You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2016/04/14 06:47:48 UTC

[01/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Repository: incubator-beam
Updated Branches:
  refs/heads/master c4cbbb1d9 -> 96765f19b


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java
deleted file mode 100644
index 2a30e86..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java
+++ /dev/null
@@ -1,326 +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 com.google.cloud.dataflow.sdk.testing;
-
-import static org.hamcrest.Matchers.greaterThan;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-
-import org.joda.time.Instant;
-import org.joda.time.ReadableInstant;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * A utility class for testing {@link WindowFn}s.
- */
-public class WindowFnTestUtils {
-
-  /**
-   * Creates a Set of elements to be used as expected output in
-   * {@link #runWindowFn}.
-   */
-  public static Set<String> set(long... timestamps) {
-    Set<String> result = new HashSet<>();
-    for (long timestamp : timestamps) {
-      result.add(timestampValue(timestamp));
-    }
-    return result;
-  }
-
-  /**
-   * Runs the {@link WindowFn} over the provided input, returning a map
-   * of windows to the timestamps in those windows.
-   */
-  public static <T, W extends BoundedWindow> Map<W, Set<String>> runWindowFn(
-      WindowFn<T, W> windowFn,
-      List<Long> timestamps) throws Exception {
-
-    final TestWindowSet<W, String> windowSet = new TestWindowSet<W, String>();
-    for (final Long timestamp : timestamps) {
-      for (W window : windowFn.assignWindows(
-          new TestAssignContext<T, W>(new Instant(timestamp), windowFn))) {
-        windowSet.put(window, timestampValue(timestamp));
-      }
-      windowFn.mergeWindows(new TestMergeContext<T, W>(windowSet, windowFn));
-    }
-    Map<W, Set<String>> actual = new HashMap<>();
-    for (W window : windowSet.windows()) {
-      actual.put(window, windowSet.get(window));
-    }
-    return actual;
-  }
-
-  public static <T, W extends BoundedWindow> Collection<W> assignedWindows(
-      WindowFn<T, W> windowFn, long timestamp) throws Exception {
-    return windowFn.assignWindows(new TestAssignContext<T, W>(new Instant(timestamp), windowFn));
-  }
-
-  private static String timestampValue(long timestamp) {
-    return "T" + new Instant(timestamp);
-  }
-
-  /**
-   * Test implementation of AssignContext.
-   */
-  private static class TestAssignContext<T, W extends BoundedWindow>
-      extends WindowFn<T, W>.AssignContext {
-    private Instant timestamp;
-
-    public TestAssignContext(Instant timestamp, WindowFn<T, W> windowFn) {
-      windowFn.super();
-      this.timestamp = timestamp;
-    }
-
-    @Override
-    public T element() {
-      return null;
-    }
-
-    @Override
-    public Instant timestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public Collection<? extends BoundedWindow> windows() {
-      return null;
-    }
-  }
-
-  /**
-   * Test implementation of MergeContext.
-   */
-  private static class TestMergeContext<T, W extends BoundedWindow>
-    extends WindowFn<T, W>.MergeContext {
-    private TestWindowSet<W, ?> windowSet;
-
-    public TestMergeContext(
-        TestWindowSet<W, ?> windowSet, WindowFn<T, W> windowFn) {
-      windowFn.super();
-      this.windowSet = windowSet;
-    }
-
-    @Override
-    public Collection<W> windows() {
-      return windowSet.windows();
-    }
-
-    @Override
-    public void merge(Collection<W> toBeMerged, W mergeResult) {
-      windowSet.merge(toBeMerged, mergeResult);
-    }
-  }
-
-  /**
-   * A WindowSet useful for testing WindowFns that simply
-   * collects the placed elements into multisets.
-   */
-  private static class TestWindowSet<W extends BoundedWindow, V> {
-
-    private Map<W, Set<V>> elements = new HashMap<>();
-
-    public void put(W window, V value) {
-      Set<V> all = elements.get(window);
-      if (all == null) {
-        all = new HashSet<>();
-        elements.put(window, all);
-      }
-      all.add(value);
-    }
-
-    public void merge(Collection<W> otherWindows, W window) {
-      if (otherWindows.isEmpty()) {
-        return;
-      }
-      Set<V> merged = new HashSet<>();
-      if (elements.containsKey(window) && !otherWindows.contains(window)) {
-        merged.addAll(elements.get(window));
-      }
-      for (W w : otherWindows) {
-        if (!elements.containsKey(w)) {
-          throw new IllegalArgumentException("Tried to merge a non-existent window:" + w);
-        }
-        merged.addAll(elements.get(w));
-        elements.remove(w);
-      }
-      elements.put(window, merged);
-    }
-
-    public Collection<W> windows() {
-      return elements.keySet();
-    }
-
-    // For testing.
-
-    public Set<V> get(W window) {
-      return elements.get(window);
-    }
-  }
-
-  /**
-   * Assigns the given {@code timestamp} to windows using the specified {@code windowFn}, and
-   * verifies that result of {@code windowFn.getOutputTimestamp} for each window is within the
-   * proper bound.
-   */
-  public static <T, W extends BoundedWindow> void validateNonInterferingOutputTimes(
-      WindowFn<T, W> windowFn, long timestamp) throws Exception {
-    Collection<W> windows = WindowFnTestUtils.<T, W>assignedWindows(windowFn, timestamp);
-
-    Instant instant = new Instant(timestamp);
-    for (W window : windows) {
-      Instant outputTimestamp = windowFn.getOutputTimeFn().assignOutputTime(instant, window);
-      assertFalse("getOutputTime must be greater than or equal to input timestamp",
-          outputTimestamp.isBefore(instant));
-      assertFalse("getOutputTime must be less than or equal to the max timestamp",
-          outputTimestamp.isAfter(window.maxTimestamp()));
-    }
-  }
-
-  /**
-   * Assigns the given {@code timestamp} to windows using the specified {@code windowFn}, and
-   * verifies that result of {@link WindowFn#getOutputTime windowFn.getOutputTime} for later windows
-   * (as defined by {@code maxTimestamp} won't prevent the watermark from passing the end of earlier
-   * windows.
-   *
-   * <p>This verifies that overlapping windows don't interfere at all. Depending on the
-   * {@code windowFn} this may be stricter than desired.
-   */
-  public static <T, W extends BoundedWindow> void validateGetOutputTimestamp(
-      WindowFn<T, W> windowFn, long timestamp) throws Exception {
-    Collection<W> windows = WindowFnTestUtils.<T, W>assignedWindows(windowFn, timestamp);
-    List<W> sortedWindows = new ArrayList<>(windows);
-    Collections.sort(sortedWindows, new Comparator<BoundedWindow>() {
-      @Override
-      public int compare(BoundedWindow o1, BoundedWindow o2) {
-        return o1.maxTimestamp().compareTo(o2.maxTimestamp());
-      }
-    });
-
-    Instant instant = new Instant(timestamp);
-    Instant endOfPrevious = null;
-    for (W window : sortedWindows) {
-      Instant outputTimestamp = windowFn.getOutputTimeFn().assignOutputTime(instant, window);
-      if (endOfPrevious == null) {
-        // If this is the first window, the output timestamp can be anything, as long as it is in
-        // the valid range.
-        assertFalse("getOutputTime must be greater than or equal to input timestamp",
-            outputTimestamp.isBefore(instant));
-        assertFalse("getOutputTime must be less than or equal to the max timestamp",
-            outputTimestamp.isAfter(window.maxTimestamp()));
-      } else {
-        // If this is a later window, the output timestamp must be after the end of the previous
-        // window
-        assertTrue("getOutputTime must be greater than the end of the previous window",
-            outputTimestamp.isAfter(endOfPrevious));
-        assertFalse("getOutputTime must be less than or equal to the max timestamp",
-            outputTimestamp.isAfter(window.maxTimestamp()));
-      }
-      endOfPrevious = window.maxTimestamp();
-    }
-  }
-
-  /**
-   * Verifies that later-ending merged windows from any of the timestamps hold up output of
-   * earlier-ending windows, using the provided {@link WindowFn} and {@link OutputTimeFn}.
-   *
-   * <p>Given a list of lists of timestamps, where each list is expected to merge into a single
-   * window with end times in ascending order, assigns and merges windows for each list (as though
-   * each were a separate key/user session). Then maps each timestamp in the list according to
-   * {@link OutputTimeFn#assignOutputTime outputTimeFn.assignOutputTime()} and
-   * {@link OutputTimeFn#combine outputTimeFn.combine()}.
-   *
-   * <p>Verifies that a overlapping windows do not hold each other up via the watermark.
-   */
-  public static <T, W extends IntervalWindow>
-  void validateGetOutputTimestamps(
-      WindowFn<T, W> windowFn,
-      OutputTimeFn<? super W> outputTimeFn,
-      List<List<Long>> timestampsPerWindow) throws Exception {
-
-    // Assign windows to each timestamp, then merge them, storing the merged windows in
-    // a list in corresponding order to timestampsPerWindow
-    final List<W> windows = new ArrayList<>();
-    for (List<Long> timestampsForWindow : timestampsPerWindow) {
-      final Set<W> windowsToMerge = new HashSet<>();
-
-      for (long timestamp : timestampsForWindow) {
-        windowsToMerge.addAll(
-            WindowFnTestUtils.<T, W>assignedWindows(windowFn, timestamp));
-      }
-
-      windowFn.mergeWindows(windowFn.new MergeContext() {
-        @Override
-        public Collection<W> windows() {
-          return windowsToMerge;
-        }
-
-        @Override
-        public void merge(Collection<W> toBeMerged, W mergeResult) throws Exception {
-          windows.add(mergeResult);
-        }
-      });
-    }
-
-    // Map every list of input timestamps to an output timestamp
-    final List<Instant> combinedOutputTimestamps = new ArrayList<>();
-    for (int i = 0; i < timestampsPerWindow.size(); ++i) {
-      List<Long> timestampsForWindow = timestampsPerWindow.get(i);
-      W window = windows.get(i);
-
-      List<Instant> outputInstants = new ArrayList<>();
-      for (long inputTimestamp : timestampsForWindow) {
-        outputInstants.add(outputTimeFn.assignOutputTime(new Instant(inputTimestamp), window));
-      }
-
-      combinedOutputTimestamps.add(OutputTimeFns.combineOutputTimes(outputTimeFn, outputInstants));
-    }
-
-    // Consider windows in increasing order of max timestamp; ensure the output timestamp is after
-    // the max timestamp of the previous
-    @Nullable W earlierEndingWindow = null;
-    for (int i = 0; i < windows.size(); ++i) {
-      W window = windows.get(i);
-      ReadableInstant outputTimestamp = combinedOutputTimestamps.get(i);
-
-      if (earlierEndingWindow != null) {
-        assertThat(outputTimestamp,
-            greaterThan((ReadableInstant) earlierEndingWindow.maxTimestamp()));
-      }
-
-      earlierEndingWindow = window;
-    }
-  }
-}


[48/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleUtils.java
deleted file mode 100644
index 5b98170..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleUtils.java
+++ /dev/null
@@ -1,488 +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 com.google.cloud.dataflow.examples.common;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.Bigquery.Datasets;
-import com.google.api.services.bigquery.Bigquery.Tables;
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.DatasetReference;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.api.services.pubsub.model.Topic;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Strings;
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.Uninterruptibles;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import javax.servlet.http.HttpServletResponse;
-
-/**
- * The utility class that sets up and tears down external resources, starts the Google Cloud Pub/Sub
- * injector, and cancels the streaming and the injector pipelines once the program terminates.
- *
- * <p>It is used to run Dataflow examples, such as TrafficMaxLaneFlow and TrafficRoutes.
- */
-public class DataflowExampleUtils {
-
-  private final DataflowPipelineOptions options;
-  private Bigquery bigQueryClient = null;
-  private Pubsub pubsubClient = null;
-  private Dataflow dataflowClient = null;
-  private Set<DataflowPipelineJob> jobsToCancel = Sets.newHashSet();
-  private List<String> pendingMessages = Lists.newArrayList();
-
-  public DataflowExampleUtils(DataflowPipelineOptions options) {
-    this.options = options;
-  }
-
-  /**
-   * Do resources and runner options setup.
-   */
-  public DataflowExampleUtils(DataflowPipelineOptions options, boolean isUnbounded)
-      throws IOException {
-    this.options = options;
-    setupResourcesAndRunner(isUnbounded);
-  }
-
-  /**
-   * Sets up external resources that are required by the example,
-   * such as Pub/Sub topics and BigQuery tables.
-   *
-   * @throws IOException if there is a problem setting up the resources
-   */
-  public void setup() throws IOException {
-    Sleeper sleeper = Sleeper.DEFAULT;
-    BackOff backOff = new AttemptBoundedExponentialBackOff(3, 200);
-    Throwable lastException = null;
-    try {
-      do {
-        try {
-          setupPubsub();
-          setupBigQueryTable();
-          return;
-        } catch (GoogleJsonResponseException e) {
-          lastException = e;
-        }
-      } while (BackOffUtils.next(sleeper, backOff));
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      // Ignore InterruptedException
-    }
-    Throwables.propagate(lastException);
-  }
-
-  /**
-   * Set up external resources, and configure the runner appropriately.
-   */
-  public void setupResourcesAndRunner(boolean isUnbounded) throws IOException {
-    if (isUnbounded) {
-      options.setStreaming(true);
-    }
-    setup();
-    setupRunner();
-  }
-
-  /**
-   * Sets up the Google Cloud Pub/Sub topic.
-   *
-   * <p>If the topic doesn't exist, a new topic with the given name will be created.
-   *
-   * @throws IOException if there is a problem setting up the Pub/Sub topic
-   */
-  public void setupPubsub() throws IOException {
-    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
-        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
-    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
-      pendingMessages.add("**********************Set Up Pubsub************************");
-      setupPubsubTopic(pubsubOptions.getPubsubTopic());
-      pendingMessages.add("The Pub/Sub topic has been set up for this example: "
-          + pubsubOptions.getPubsubTopic());
-
-      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
-        setupPubsubSubscription(
-            pubsubOptions.getPubsubTopic(), pubsubOptions.getPubsubSubscription());
-        pendingMessages.add("The Pub/Sub subscription has been set up for this example: "
-            + pubsubOptions.getPubsubSubscription());
-      }
-    }
-  }
-
-  /**
-   * Sets up the BigQuery table with the given schema.
-   *
-   * <p>If the table already exists, the schema has to match the given one. Otherwise, the example
-   * will throw a RuntimeException. If the table doesn't exist, a new table with the given schema
-   * will be created.
-   *
-   * @throws IOException if there is a problem setting up the BigQuery table
-   */
-  public void setupBigQueryTable() throws IOException {
-    ExampleBigQueryTableOptions bigQueryTableOptions =
-        options.as(ExampleBigQueryTableOptions.class);
-    if (bigQueryTableOptions.getBigQueryDataset() != null
-        && bigQueryTableOptions.getBigQueryTable() != null
-        && bigQueryTableOptions.getBigQuerySchema() != null) {
-      pendingMessages.add("******************Set Up Big Query Table*******************");
-      setupBigQueryTable(bigQueryTableOptions.getProject(),
-                         bigQueryTableOptions.getBigQueryDataset(),
-                         bigQueryTableOptions.getBigQueryTable(),
-                         bigQueryTableOptions.getBigQuerySchema());
-      pendingMessages.add("The BigQuery table has been set up for this example: "
-          + bigQueryTableOptions.getProject()
-          + ":" + bigQueryTableOptions.getBigQueryDataset()
-          + "." + bigQueryTableOptions.getBigQueryTable());
-    }
-  }
-
-  /**
-   * Tears down external resources that can be deleted upon the example's completion.
-   */
-  private void tearDown() {
-    pendingMessages.add("*************************Tear Down*************************");
-    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
-        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
-    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
-      try {
-        deletePubsubTopic(pubsubOptions.getPubsubTopic());
-        pendingMessages.add("The Pub/Sub topic has been deleted: "
-            + pubsubOptions.getPubsubTopic());
-      } catch (IOException e) {
-        pendingMessages.add("Failed to delete the Pub/Sub topic : "
-            + pubsubOptions.getPubsubTopic());
-      }
-      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
-        try {
-          deletePubsubSubscription(pubsubOptions.getPubsubSubscription());
-          pendingMessages.add("The Pub/Sub subscription has been deleted: "
-              + pubsubOptions.getPubsubSubscription());
-        } catch (IOException e) {
-          pendingMessages.add("Failed to delete the Pub/Sub subscription : "
-              + pubsubOptions.getPubsubSubscription());
-        }
-      }
-    }
-
-    ExampleBigQueryTableOptions bigQueryTableOptions =
-        options.as(ExampleBigQueryTableOptions.class);
-    if (bigQueryTableOptions.getBigQueryDataset() != null
-        && bigQueryTableOptions.getBigQueryTable() != null
-        && bigQueryTableOptions.getBigQuerySchema() != null) {
-      pendingMessages.add("The BigQuery table might contain the example's output, "
-          + "and it is not deleted automatically: "
-          + bigQueryTableOptions.getProject()
-          + ":" + bigQueryTableOptions.getBigQueryDataset()
-          + "." + bigQueryTableOptions.getBigQueryTable());
-      pendingMessages.add("Please go to the Developers Console to delete it manually."
-          + " Otherwise, you may be charged for its usage.");
-    }
-  }
-
-  private void setupBigQueryTable(String projectId, String datasetId, String tableId,
-      TableSchema schema) throws IOException {
-    if (bigQueryClient == null) {
-      bigQueryClient = Transport.newBigQueryClient(options.as(BigQueryOptions.class)).build();
-    }
-
-    Datasets datasetService = bigQueryClient.datasets();
-    if (executeNullIfNotFound(datasetService.get(projectId, datasetId)) == null) {
-      Dataset newDataset = new Dataset().setDatasetReference(
-          new DatasetReference().setProjectId(projectId).setDatasetId(datasetId));
-      datasetService.insert(projectId, newDataset).execute();
-    }
-
-    Tables tableService = bigQueryClient.tables();
-    Table table = executeNullIfNotFound(tableService.get(projectId, datasetId, tableId));
-    if (table == null) {
-      Table newTable = new Table().setSchema(schema).setTableReference(
-          new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId));
-      tableService.insert(projectId, datasetId, newTable).execute();
-    } else if (!table.getSchema().equals(schema)) {
-      throw new RuntimeException(
-          "Table exists and schemas do not match, expecting: " + schema.toPrettyString()
-          + ", actual: " + table.getSchema().toPrettyString());
-    }
-  }
-
-  private void setupPubsubTopic(String topic) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) == null) {
-      pubsubClient.projects().topics().create(topic, new Topic().setName(topic)).execute();
-    }
-  }
-
-  private void setupPubsubSubscription(String topic, String subscription) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) == null) {
-      Subscription subInfo = new Subscription()
-        .setAckDeadlineSeconds(60)
-        .setTopic(topic);
-      pubsubClient.projects().subscriptions().create(subscription, subInfo).execute();
-    }
-  }
-
-  /**
-   * Deletes the Google Cloud Pub/Sub topic.
-   *
-   * @throws IOException if there is a problem deleting the Pub/Sub topic
-   */
-  private void deletePubsubTopic(String topic) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) != null) {
-      pubsubClient.projects().topics().delete(topic).execute();
-    }
-  }
-
-  /**
-   * Deletes the Google Cloud Pub/Sub subscription.
-   *
-   * @throws IOException if there is a problem deleting the Pub/Sub subscription
-   */
-  private void deletePubsubSubscription(String subscription) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) != null) {
-      pubsubClient.projects().subscriptions().delete(subscription).execute();
-    }
-  }
-
-  /**
-   * If this is an unbounded (streaming) pipeline, and both inputFile and pubsub topic are defined,
-   * start an 'injector' pipeline that publishes the contents of the file to the given topic, first
-   * creating the topic if necessary.
-   */
-  public void startInjectorIfNeeded(String inputFile) {
-    ExamplePubsubTopicOptions pubsubTopicOptions = options.as(ExamplePubsubTopicOptions.class);
-    if (pubsubTopicOptions.isStreaming()
-        && !Strings.isNullOrEmpty(inputFile)
-        && !Strings.isNullOrEmpty(pubsubTopicOptions.getPubsubTopic())) {
-      runInjectorPipeline(inputFile, pubsubTopicOptions.getPubsubTopic());
-    }
-  }
-
-  /**
-   * Do some runner setup: check that the DirectPipelineRunner is not used in conjunction with
-   * streaming, and if streaming is specified, use the DataflowPipelineRunner. Return the streaming
-   * flag value.
-   */
-  public void setupRunner() {
-    if (options.isStreaming() && options.getRunner() != DirectPipelineRunner.class) {
-      // In order to cancel the pipelines automatically,
-      // {@literal DataflowPipelineRunner} is forced to be used.
-      options.setRunner(DataflowPipelineRunner.class);
-    }
-  }
-
-  /**
-   * Runs a batch pipeline to inject data into the PubSubIO input topic.
-   *
-   * <p>The injector pipeline will read from the given text file, and inject data
-   * into the Google Cloud Pub/Sub topic.
-   */
-  public void runInjectorPipeline(String inputFile, String topic) {
-    runInjectorPipeline(TextIO.Read.from(inputFile), topic, null);
-  }
-
-  /**
-   * Runs a batch pipeline to inject data into the PubSubIO input topic.
-   *
-   * <p>The injector pipeline will read from the given source, and inject data
-   * into the Google Cloud Pub/Sub topic.
-   */
-  public void runInjectorPipeline(PTransform<? super PBegin, PCollection<String>> readSource,
-                                  String topic,
-                                  String pubsubTimestampTabelKey) {
-    PubsubFileInjector.Bound injector;
-    if (Strings.isNullOrEmpty(pubsubTimestampTabelKey)) {
-      injector = PubsubFileInjector.publish(topic);
-    } else {
-      injector = PubsubFileInjector.withTimestampLabelKey(pubsubTimestampTabelKey).publish(topic);
-    }
-    DataflowPipelineOptions copiedOptions = options.cloneAs(DataflowPipelineOptions.class);
-    if (options.getServiceAccountName() != null) {
-      copiedOptions.setServiceAccountName(options.getServiceAccountName());
-    }
-    if (options.getServiceAccountKeyfile() != null) {
-      copiedOptions.setServiceAccountKeyfile(options.getServiceAccountKeyfile());
-    }
-    copiedOptions.setStreaming(false);
-    copiedOptions.setWorkerHarnessContainerImage(
-        DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
-    copiedOptions.setNumWorkers(options.as(DataflowExampleOptions.class).getInjectorNumWorkers());
-    copiedOptions.setJobName(options.getJobName() + "-injector");
-    Pipeline injectorPipeline = Pipeline.create(copiedOptions);
-    injectorPipeline.apply(readSource)
-                    .apply(IntraBundleParallelization
-                        .of(injector)
-                        .withMaxParallelism(20));
-    PipelineResult result = injectorPipeline.run();
-    if (result instanceof DataflowPipelineJob) {
-      jobsToCancel.add(((DataflowPipelineJob) result));
-    }
-  }
-
-  /**
-   * Runs the provided pipeline to inject data into the PubSubIO input topic.
-   */
-  public void runInjectorPipeline(Pipeline injectorPipeline) {
-    PipelineResult result = injectorPipeline.run();
-    if (result instanceof DataflowPipelineJob) {
-      jobsToCancel.add(((DataflowPipelineJob) result));
-    }
-  }
-
-  /**
-   * Start the auxiliary injector pipeline, then wait for this pipeline to finish.
-   */
-  public void mockUnboundedSource(String inputFile, PipelineResult result) {
-    startInjectorIfNeeded(inputFile);
-    waitToFinish(result);
-  }
-
-  /**
-   * If {@literal DataflowPipelineRunner} or {@literal BlockingDataflowPipelineRunner} is used,
-   * waits for the pipeline to finish and cancels it (and the injector) before the program exists.
-   */
-  public void waitToFinish(PipelineResult result) {
-    if (result instanceof DataflowPipelineJob) {
-      final DataflowPipelineJob job = (DataflowPipelineJob) result;
-      jobsToCancel.add(job);
-      if (!options.as(DataflowExampleOptions.class).getKeepJobsRunning()) {
-        addShutdownHook(jobsToCancel);
-      }
-      try {
-        job.waitToFinish(-1, TimeUnit.SECONDS, new MonitoringUtil.PrintHandler(System.out));
-      } catch (Exception e) {
-        throw new RuntimeException("Failed to wait for job to finish: " + job.getJobId());
-      }
-    } else {
-      // Do nothing if the given PipelineResult doesn't support waitToFinish(),
-      // such as EvaluationResults returned by DirectPipelineRunner.
-      tearDown();
-      printPendingMessages();
-    }
-  }
-
-  private void addShutdownHook(final Collection<DataflowPipelineJob> jobs) {
-    if (dataflowClient == null) {
-      dataflowClient = options.getDataflowClient();
-    }
-
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      @Override
-      public void run() {
-        tearDown();
-        printPendingMessages();
-        for (DataflowPipelineJob job : jobs) {
-          System.out.println("Canceling example pipeline: " + job.getJobId());
-          try {
-            job.cancel();
-          } catch (IOException e) {
-            System.out.println("Failed to cancel the job,"
-                + " please go to the Developers Console to cancel it manually");
-            System.out.println(
-                MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId()));
-          }
-        }
-
-        for (DataflowPipelineJob job : jobs) {
-          boolean cancellationVerified = false;
-          for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) {
-            if (job.getState().isTerminal()) {
-              cancellationVerified = true;
-              System.out.println("Canceled example pipeline: " + job.getJobId());
-              break;
-            } else {
-              System.out.println(
-                  "The example pipeline is still running. Verifying the cancellation.");
-            }
-            Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS);
-          }
-          if (!cancellationVerified) {
-            System.out.println("Failed to verify the cancellation for job: " + job.getJobId());
-            System.out.println("Please go to the Developers Console to verify manually:");
-            System.out.println(
-                MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId()));
-          }
-        }
-      }
-    });
-  }
-
-  private void printPendingMessages() {
-    System.out.println();
-    System.out.println("***********************************************************");
-    System.out.println("***********************************************************");
-    for (String message : pendingMessages) {
-      System.out.println(message);
-    }
-    System.out.println("***********************************************************");
-    System.out.println("***********************************************************");
-  }
-
-  private static <T> T executeNullIfNotFound(
-      AbstractGoogleClientRequest<T> request) throws IOException {
-    try {
-      return request.execute();
-    } catch (GoogleJsonResponseException e) {
-      if (e.getStatusCode() == HttpServletResponse.SC_NOT_FOUND) {
-        return null;
-      } else {
-        throw e;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExampleBigQueryTableOptions.java
deleted file mode 100644
index a026f3e..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExampleBigQueryTableOptions.java
+++ /dev/null
@@ -1,56 +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 com.google.cloud.dataflow.examples.common;
-
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure BigQuery tables in Dataflow examples.
- * The project defaults to the project being used to run the example.
- */
-public interface ExampleBigQueryTableOptions extends DataflowPipelineOptions {
-  @Description("BigQuery dataset name")
-  @Default.String("dataflow_examples")
-  String getBigQueryDataset();
-  void setBigQueryDataset(String dataset);
-
-  @Description("BigQuery table name")
-  @Default.InstanceFactory(BigQueryTableFactory.class)
-  String getBigQueryTable();
-  void setBigQueryTable(String table);
-
-  @Description("BigQuery table schema")
-  TableSchema getBigQuerySchema();
-  void setBigQuerySchema(TableSchema schema);
-
-  /**
-   * Returns the job name as the default BigQuery table name.
-   */
-  static class BigQueryTableFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      return options.as(DataflowPipelineOptions.class).getJobName()
-          .replace('-', '_');
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
deleted file mode 100644
index bd8cace..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicAndSubscriptionOptions.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 com.google.cloud.dataflow.examples.common;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure Pub/Sub topic/subscription in Dataflow examples.
- */
-public interface ExamplePubsubTopicAndSubscriptionOptions extends ExamplePubsubTopicOptions {
-  @Description("Pub/Sub subscription")
-  @Default.InstanceFactory(PubsubSubscriptionFactory.class)
-  String getPubsubSubscription();
-  void setPubsubSubscription(String subscription);
-
-  /**
-   * Returns a default Pub/Sub subscription based on the project and the job names.
-   */
-  static class PubsubSubscriptionFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      DataflowPipelineOptions dataflowPipelineOptions =
-          options.as(DataflowPipelineOptions.class);
-      return "projects/" + dataflowPipelineOptions.getProject()
-          + "/subscriptions/" + dataflowPipelineOptions.getJobName();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicOptions.java
deleted file mode 100644
index 14df8d9..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/ExamplePubsubTopicOptions.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 com.google.cloud.dataflow.examples.common;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure Pub/Sub topic in Dataflow examples.
- */
-public interface ExamplePubsubTopicOptions extends DataflowPipelineOptions {
-  @Description("Pub/Sub topic")
-  @Default.InstanceFactory(PubsubTopicFactory.class)
-  String getPubsubTopic();
-  void setPubsubTopic(String topic);
-
-  /**
-   * Returns a default Pub/Sub topic based on the project and the job names.
-   */
-  static class PubsubTopicFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      DataflowPipelineOptions dataflowPipelineOptions =
-          options.as(DataflowPipelineOptions.class);
-      return "projects/" + dataflowPipelineOptions.getProject()
-          + "/topics/" + dataflowPipelineOptions.getJobName();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/PubsubFileInjector.java
deleted file mode 100644
index eb3d1ac..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/PubsubFileInjector.java
+++ /dev/null
@@ -1,154 +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 com.google.cloud.dataflow.examples.common;
-
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.common.collect.ImmutableMap;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * A batch Dataflow pipeline for injecting a set of GCS files into
- * a PubSub topic line by line. Empty lines are skipped.
- *
- * <p>This is useful for testing streaming
- * pipelines. Note that since batch pipelines might retry chunks, this
- * does _not_ guarantee exactly-once injection of file data. Some lines may
- * be published multiple times.
- * </p>
- */
-public class PubsubFileInjector {
-
-  /**
-   * An incomplete {@code PubsubFileInjector} transform with unbound output topic.
-   */
-  public static class Unbound {
-    private final String timestampLabelKey;
-
-    Unbound() {
-      this.timestampLabelKey = null;
-    }
-
-    Unbound(String timestampLabelKey) {
-      this.timestampLabelKey = timestampLabelKey;
-    }
-
-    Unbound withTimestampLabelKey(String timestampLabelKey) {
-      return new Unbound(timestampLabelKey);
-    }
-
-    public Bound publish(String outputTopic) {
-      return new Bound(outputTopic, timestampLabelKey);
-    }
-  }
-
-  /** A DoFn that publishes non-empty lines to Google Cloud PubSub. */
-  public static class Bound extends DoFn<String, Void> {
-    private final String outputTopic;
-    private final String timestampLabelKey;
-    public transient Pubsub pubsub;
-
-    public Bound(String outputTopic, String timestampLabelKey) {
-      this.outputTopic = outputTopic;
-      this.timestampLabelKey = timestampLabelKey;
-    }
-
-    @Override
-    public void startBundle(Context context) {
-      this.pubsub =
-          Transport.newPubsubClient(context.getPipelineOptions().as(DataflowPipelineOptions.class))
-              .build();
-    }
-
-    @Override
-    public void processElement(ProcessContext c) throws IOException {
-      if (c.element().isEmpty()) {
-        return;
-      }
-      PubsubMessage pubsubMessage = new PubsubMessage();
-      pubsubMessage.encodeData(c.element().getBytes());
-      if (timestampLabelKey != null) {
-        pubsubMessage.setAttributes(
-            ImmutableMap.of(timestampLabelKey, Long.toString(c.timestamp().getMillis())));
-      }
-      PublishRequest publishRequest = new PublishRequest();
-      publishRequest.setMessages(Arrays.asList(pubsubMessage));
-      this.pubsub.projects().topics().publish(outputTopic, publishRequest).execute();
-    }
-  }
-
-  /**
-   * Creates a {@code PubsubFileInjector} transform with the given timestamp label key.
-   */
-  public static Unbound withTimestampLabelKey(String timestampLabelKey) {
-    return new Unbound(timestampLabelKey);
-  }
-
-  /**
-   * Creates a {@code PubsubFileInjector} transform that publishes to the given output topic.
-   */
-  public static Bound publish(String outputTopic) {
-    return new Unbound().publish(outputTopic);
-  }
-
-  /**
-   * Command line parameter options.
-   */
-  private interface PubsubFileInjectorOptions extends PipelineOptions {
-    @Description("GCS location of files.")
-    @Validation.Required
-    String getInput();
-    void setInput(String value);
-
-    @Description("Topic to publish on.")
-    @Validation.Required
-    String getOutputTopic();
-    void setOutputTopic(String value);
-  }
-
-  /**
-   * Sets up and starts streaming pipeline.
-   */
-  public static void main(String[] args) {
-    PubsubFileInjectorOptions options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(PubsubFileInjectorOptions.class);
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    pipeline
-        .apply(TextIO.Read.from(options.getInput()))
-        .apply(IntraBundleParallelization.of(PubsubFileInjector.publish(options.getOutputTopic()))
-            .withMaxParallelism(20));
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java
deleted file mode 100644
index 343b17b..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java
+++ /dev/null
@@ -1,517 +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 com.google.cloud.dataflow.examples.complete;
-
-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.api.services.datastore.DatastoreV1.Entity;
-import com.google.api.services.datastore.DatastoreV1.Key;
-import com.google.api.services.datastore.DatastoreV1.Value;
-import com.google.api.services.datastore.client.DatastoreHelper;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Top;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-
-import org.joda.time.Duration;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * An example that computes the most popular hash tags
- * for every prefix, which can be used for auto-completion.
- *
- * <p>Concepts: Using the same pipeline in both streaming and batch, combiners,
- *              composite transforms.
- *
- * <p>To execute this pipeline using the Dataflow service in batch mode,
- * specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=DataflowPipelineRunner
- *   --inputFile=gs://path/to/input*.txt
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service in streaming mode,
- * specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=DataflowPipelineRunner
- *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
- *   --streaming
- * }</pre>
- *
- * <p>This will update the datastore every 10 seconds based on the last
- * 30 minutes of data received.
- */
-public class AutoComplete {
-
-  /**
-   * A PTransform that takes as input a list of tokens and returns
-   * the most common tokens per prefix.
-   */
-  public static class ComputeTopCompletions
-      extends PTransform<PCollection<String>, PCollection<KV<String, List<CompletionCandidate>>>> {
-    private final int candidatesPerPrefix;
-    private final boolean recursive;
-
-    protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) {
-      this.candidatesPerPrefix = candidatesPerPrefix;
-      this.recursive = recursive;
-    }
-
-    public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) {
-      return new ComputeTopCompletions(candidatesPerPrefix, recursive);
-    }
-
-    @Override
-    public PCollection<KV<String, List<CompletionCandidate>>> apply(PCollection<String> input) {
-      PCollection<CompletionCandidate> candidates = input
-        // First count how often each token appears.
-        .apply(new Count.PerElement<String>())
-
-        // Map the KV outputs of Count into our own CompletionCandiate class.
-        .apply(ParDo.named("CreateCompletionCandidates").of(
-            new DoFn<KV<String, Long>, CompletionCandidate>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.output(new CompletionCandidate(c.element().getKey(), c.element().getValue()));
-              }
-            }));
-
-      // Compute the top via either a flat or recursive algorithm.
-      if (recursive) {
-        return candidates
-          .apply(new ComputeTopRecursive(candidatesPerPrefix, 1))
-          .apply(Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
-      } else {
-        return candidates
-          .apply(new ComputeTopFlat(candidatesPerPrefix, 1));
-      }
-    }
-  }
-
-  /**
-   * Lower latency, but more expensive.
-   */
-  private static class ComputeTopFlat
-      extends PTransform<PCollection<CompletionCandidate>,
-                         PCollection<KV<String, List<CompletionCandidate>>>> {
-    private final int candidatesPerPrefix;
-    private final int minPrefix;
-
-    public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) {
-      this.candidatesPerPrefix = candidatesPerPrefix;
-      this.minPrefix = minPrefix;
-    }
-
-    @Override
-    public PCollection<KV<String, List<CompletionCandidate>>> apply(
-        PCollection<CompletionCandidate> input) {
-      return input
-        // For each completion candidate, map it to all prefixes.
-        .apply(ParDo.of(new AllPrefixes(minPrefix)))
-
-        // Find and return the top candiates for each prefix.
-        .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix)
-               .withHotKeyFanout(new HotKeyFanout()));
-    }
-
-    private static class HotKeyFanout implements SerializableFunction<String, Integer> {
-      @Override
-      public Integer apply(String input) {
-        return (int) Math.pow(4, 5 - input.length());
-      }
-    }
-  }
-
-  /**
-   * Cheaper but higher latency.
-   *
-   * <p>Returns two PCollections, the first is top prefixes of size greater
-   * than minPrefix, and the second is top prefixes of size exactly
-   * minPrefix.
-   */
-  private static class ComputeTopRecursive
-      extends PTransform<PCollection<CompletionCandidate>,
-                         PCollectionList<KV<String, List<CompletionCandidate>>>> {
-    private final int candidatesPerPrefix;
-    private final int minPrefix;
-
-    public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) {
-      this.candidatesPerPrefix = candidatesPerPrefix;
-      this.minPrefix = minPrefix;
-    }
-
-    private class KeySizePartitionFn implements PartitionFn<KV<String, List<CompletionCandidate>>> {
-      @Override
-      public int partitionFor(KV<String, List<CompletionCandidate>> elem, int numPartitions) {
-        return elem.getKey().length() > minPrefix ? 0 : 1;
-      }
-    }
-
-    private static class FlattenTops
-        extends DoFn<KV<String, List<CompletionCandidate>>, CompletionCandidate> {
-      @Override
-      public void processElement(ProcessContext c) {
-        for (CompletionCandidate cc : c.element().getValue()) {
-          c.output(cc);
-        }
-      }
-    }
-
-    @Override
-    public PCollectionList<KV<String, List<CompletionCandidate>>> apply(
-          PCollection<CompletionCandidate> input) {
-        if (minPrefix > 10) {
-          // Base case, partitioning to return the output in the expected format.
-          return input
-            .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix))
-            .apply(Partition.of(2, new KeySizePartitionFn()));
-        } else {
-          // If a candidate is in the top N for prefix a...b, it must also be in the top
-          // N for a...bX for every X, which is typlically a much smaller set to consider.
-          // First, compute the top candidate for prefixes of size at least minPrefix + 1.
-          PCollectionList<KV<String, List<CompletionCandidate>>> larger = input
-            .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1));
-          // Consider the top candidates for each prefix of length minPrefix + 1...
-          PCollection<KV<String, List<CompletionCandidate>>> small =
-            PCollectionList
-            .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
-            // ...together with those (previously excluded) candidates of length
-            // exactly minPrefix...
-            .and(input.apply(Filter.byPredicate(
-                new SerializableFunction<CompletionCandidate, Boolean>() {
-                  @Override
-                  public Boolean apply(CompletionCandidate c) {
-                    return c.getValue().length() == minPrefix;
-                  }
-                })))
-            .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
-            // ...set the key to be the minPrefix-length prefix...
-            .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))
-            // ...and (re)apply the Top operator to all of them together.
-            .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix));
-
-          PCollection<KV<String, List<CompletionCandidate>>> flattenLarger = larger
-              .apply("FlattenLarge", Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
-
-          return PCollectionList.of(flattenLarger).and(small);
-        }
-    }
-  }
-
-  /**
-   * A DoFn that keys each candidate by all its prefixes.
-   */
-  private static class AllPrefixes
-      extends DoFn<CompletionCandidate, KV<String, CompletionCandidate>> {
-    private final int minPrefix;
-    private final int maxPrefix;
-    public AllPrefixes(int minPrefix) {
-      this(minPrefix, Integer.MAX_VALUE);
-    }
-    public AllPrefixes(int minPrefix, int maxPrefix) {
-      this.minPrefix = minPrefix;
-      this.maxPrefix = maxPrefix;
-    }
-    @Override
-      public void processElement(ProcessContext c) {
-      String word = c.element().value;
-      for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
-        c.output(KV.of(word.substring(0, i), c.element()));
-      }
-    }
-  }
-
-  /**
-   * Class used to store tag-count pairs.
-   */
-  @DefaultCoder(AvroCoder.class)
-  static class CompletionCandidate implements Comparable<CompletionCandidate> {
-    private long count;
-    private String value;
-
-    public CompletionCandidate(String value, long count) {
-      this.value = value;
-      this.count = count;
-    }
-
-    public long getCount() {
-      return count;
-    }
-
-    public String getValue() {
-      return value;
-    }
-
-    // Empty constructor required for Avro decoding.
-    public CompletionCandidate() {}
-
-    @Override
-    public int compareTo(CompletionCandidate o) {
-      if (this.count < o.count) {
-        return -1;
-      } else if (this.count == o.count) {
-        return this.value.compareTo(o.value);
-      } else {
-        return 1;
-      }
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other instanceof CompletionCandidate) {
-        CompletionCandidate that = (CompletionCandidate) other;
-        return this.count == that.count && this.value.equals(that.value);
-      } else {
-        return false;
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      return Long.valueOf(count).hashCode() ^ value.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return "CompletionCandidate[" + value + ", " + count + "]";
-    }
-  }
-
-  /**
-   * Takes as input a set of strings, and emits each #hashtag found therein.
-   */
-  static class ExtractHashtags extends DoFn<String, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      Matcher m = Pattern.compile("#\\S+").matcher(c.element());
-      while (m.find()) {
-        c.output(m.group().substring(1));
-      }
-    }
-  }
-
-  static class FormatForBigquery extends DoFn<KV<String, List<CompletionCandidate>>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      List<TableRow> completions = new ArrayList<>();
-      for (CompletionCandidate cc : c.element().getValue()) {
-        completions.add(new TableRow()
-          .set("count", cc.getCount())
-          .set("tag", cc.getValue()));
-      }
-      TableRow row = new TableRow()
-        .set("prefix", c.element().getKey())
-        .set("tags", completions);
-      c.output(row);
-    }
-
-    /**
-     * Defines the BigQuery schema used for the output.
-     */
-    static TableSchema getSchema() {
-      List<TableFieldSchema> tagFields = new ArrayList<>();
-      tagFields.add(new TableFieldSchema().setName("count").setType("INTEGER"));
-      tagFields.add(new TableFieldSchema().setName("tag").setType("STRING"));
-      List<TableFieldSchema> fields = new ArrayList<>();
-      fields.add(new TableFieldSchema().setName("prefix").setType("STRING"));
-      fields.add(new TableFieldSchema()
-          .setName("tags").setType("RECORD").setMode("REPEATED").setFields(tagFields));
-      return new TableSchema().setFields(fields);
-    }
-  }
-
-  /**
-   * Takes as input a the top candidates per prefix, and emits an entity
-   * suitable for writing to Datastore.
-   */
-  static class FormatForDatastore extends DoFn<KV<String, List<CompletionCandidate>>, Entity> {
-    private String kind;
-
-    public FormatForDatastore(String kind) {
-      this.kind = kind;
-    }
-
-    @Override
-    public void processElement(ProcessContext c) {
-      Entity.Builder entityBuilder = Entity.newBuilder();
-      Key key = DatastoreHelper.makeKey(kind, c.element().getKey()).build();
-
-      entityBuilder.setKey(key);
-      List<Value> candidates = new ArrayList<>();
-      for (CompletionCandidate tag : c.element().getValue()) {
-        Entity.Builder tagEntity = Entity.newBuilder();
-        tagEntity.addProperty(
-            DatastoreHelper.makeProperty("tag", DatastoreHelper.makeValue(tag.value)));
-        tagEntity.addProperty(
-            DatastoreHelper.makeProperty("count", DatastoreHelper.makeValue(tag.count)));
-        candidates.add(DatastoreHelper.makeValue(tagEntity).setIndexed(false).build());
-      }
-      entityBuilder.addProperty(
-          DatastoreHelper.makeProperty("candidates", DatastoreHelper.makeValue(candidates)));
-      c.output(entityBuilder.build());
-    }
-  }
-
-  /**
-   * Options supported by this class.
-   *
-   * <p>Inherits standard Dataflow configuration options.
-   */
-  private static interface Options extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
-    @Description("Input text file")
-    String getInputFile();
-    void setInputFile(String value);
-
-    @Description("Whether to use the recursive algorithm")
-    @Default.Boolean(true)
-    Boolean getRecursive();
-    void setRecursive(Boolean value);
-
-    @Description("Dataset entity kind")
-    @Default.String("autocomplete-demo")
-    String getKind();
-    void setKind(String value);
-
-    @Description("Whether output to BigQuery")
-    @Default.Boolean(true)
-    Boolean getOutputToBigQuery();
-    void setOutputToBigQuery(Boolean value);
-
-    @Description("Whether output to Datastore")
-    @Default.Boolean(false)
-    Boolean getOutputToDatastore();
-    void setOutputToDatastore(Boolean value);
-
-    @Description("Datastore output dataset ID, defaults to project ID")
-    String getOutputDataset();
-    void setOutputDataset(String value);
-  }
-
-  public static void main(String[] args) throws IOException {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
-    if (options.isStreaming()) {
-      // In order to cancel the pipelines automatically,
-      // {@literal DataflowPipelineRunner} is forced to be used.
-      options.setRunner(DataflowPipelineRunner.class);
-    }
-
-    options.setBigQuerySchema(FormatForBigquery.getSchema());
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
-
-    // We support running the same pipeline in either
-    // batch or windowed streaming mode.
-    PTransform<? super PBegin, PCollection<String>> readSource;
-    WindowFn<Object, ?> windowFn;
-    if (options.isStreaming()) {
-      Preconditions.checkArgument(
-          !options.getOutputToDatastore(), "DatastoreIO is not supported in streaming.");
-      dataflowUtils.setupPubsub();
-
-      readSource = PubsubIO.Read.topic(options.getPubsubTopic());
-      windowFn = SlidingWindows.of(Duration.standardMinutes(30)).every(Duration.standardSeconds(5));
-    } else {
-      readSource = TextIO.Read.from(options.getInputFile());
-      windowFn = new GlobalWindows();
-    }
-
-    // Create the pipeline.
-    Pipeline p = Pipeline.create(options);
-    PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
-      .apply(readSource)
-      .apply(ParDo.of(new ExtractHashtags()))
-      .apply(Window.<String>into(windowFn))
-      .apply(ComputeTopCompletions.top(10, options.getRecursive()));
-
-    if (options.getOutputToDatastore()) {
-      toWrite
-      .apply(ParDo.named("FormatForDatastore").of(new FormatForDatastore(options.getKind())))
-      .apply(DatastoreIO.writeTo(MoreObjects.firstNonNull(
-          options.getOutputDataset(), options.getProject())));
-    }
-    if (options.getOutputToBigQuery()) {
-      dataflowUtils.setupBigQueryTable();
-
-      TableReference tableRef = new TableReference();
-      tableRef.setProjectId(options.getProject());
-      tableRef.setDatasetId(options.getBigQueryDataset());
-      tableRef.setTableId(options.getBigQueryTable());
-
-      toWrite
-        .apply(ParDo.of(new FormatForBigquery()))
-        .apply(BigQueryIO.Write
-               .to(tableRef)
-               .withSchema(FormatForBigquery.getSchema())
-               .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-               .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
-    }
-
-    // Run the pipeline.
-    PipelineResult result = p.run();
-
-    if (options.isStreaming() && !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.waitToFinish(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/README.md b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/README.md
deleted file mode 100644
index 5fba154..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/README.md
+++ /dev/null
@@ -1,44 +0,0 @@
-
-# "Complete" Examples
-
-This directory contains end-to-end example pipelines that perform complex data processing tasks. They include:
-
-<ul>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java">AutoComplete</a>
-  &mdash; An example that computes the most popular hash tags for every
-  prefix, which can be used for auto-completion. Demonstrates how to use the
-  same pipeline in both streaming and batch, combiners, and composite
-  transforms.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java">StreamingWordExtract</a>
-  &mdash; A streaming pipeline example that inputs lines of text from a Cloud
-  Pub/Sub topic, splits each line into individual words, capitalizes those
-  words, and writes the output to a BigQuery table.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java">TfIdf</a>
-  &mdash; An example that computes a basic TF-IDF search table for a directory or
-  Cloud Storage prefix. Demonstrates joining data, side inputs, and logging.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java">TopWikipediaSessions</a>
-  &mdash; An example that reads Wikipedia edit data from Cloud Storage and
-  computes the user with the longest string of edits separated by no more than
-  an hour within each month. Demonstrates using Cloud Dataflow
-  <code>Windowing</code> to perform time-based aggregations of data.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java">TrafficMaxLaneFlow</a>
-  &mdash; A streaming Cloud Dataflow example using BigQuery output in the
-  <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
-  runner, sliding windows, Cloud Pub/Sub topic ingestion, the use of the
-  <code>AvroCoder</code> to encode a custom class, and custom
-  <code>Combine</code> transforms.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java">TrafficRoutes</a>
-  &mdash; A streaming Cloud Dataflow example using BigQuery output in the
-  <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
-  runner, <code>GroupByKey</code>, keyed state, sliding windows, and Cloud
-  Pub/Sub topic ingestion.
-  </li>
-  </ul>
-
-See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
-README](../../../../../../../../../README.md) for
-information about how to run these examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java
deleted file mode 100644
index c61873c..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java
+++ /dev/null
@@ -1,164 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-
-import java.io.IOException;
-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
- * 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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
- * and then exits.
- */
-public class StreamingWordExtract {
-
-  /** A DoFn that tokenizes lines of text into individual words. */
-  static class ExtractWords extends DoFn<String, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] words = c.element().split("[^a-zA-Z']+");
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(word);
-        }
-      }
-    }
-  }
-
-  /** A DoFn that uppercases a word. */
-  static class Uppercase extends DoFn<String, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element().toUpperCase());
-    }
-  }
-
-  /**
-   * Converts strings into BigQuery rows.
-   */
-  static class StringToRowConverter extends DoFn<String, TableRow> {
-    /**
-     * In this example, put the whole string into single BigQuery field.
-     */
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(new TableRow().set("string_field", c.element()));
-    }
-
-    static TableSchema getSchema() {
-      return new TableSchema().setFields(new ArrayList<TableFieldSchema>() {
-            // Compose the list of TableFieldSchema from tableSchema.
-            {
-              add(new TableFieldSchema().setName("string_field").setType("STRING"));
-            }
-      });
-    }
-  }
-
-  /**
-   * Options supported by {@link StreamingWordExtract}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private interface StreamingWordExtractOptions
-      extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
-    @Description("Input file to inject to Pub/Sub topic")
-    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
-    String getInputFile();
-    void setInputFile(String value);
-  }
-
-  /**
-   * Sets up and starts streaming pipeline.
-   *
-   * @throws IOException if there is a problem setting up resources
-   */
-  public static void main(String[] args) throws IOException {
-    StreamingWordExtractOptions options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(StreamingWordExtractOptions.class);
-    options.setStreaming(true);
-    // In order to cancel the pipelines automatically,
-    // {@literal DataflowPipelineRunner} is forced to be used.
-    options.setRunner(DataflowPipelineRunner.class);
-
-    options.setBigQuerySchema(StringToRowConverter.getSchema());
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
-    dataflowUtils.setup();
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    String tableSpec = new StringBuilder()
-        .append(options.getProject()).append(":")
-        .append(options.getBigQueryDataset()).append(".")
-        .append(options.getBigQueryTable())
-        .toString();
-    pipeline
-        .apply(PubsubIO.Read.topic(options.getPubsubTopic()))
-        .apply(ParDo.of(new ExtractWords()))
-        .apply(ParDo.of(new Uppercase()))
-        .apply(ParDo.of(new StringToRowConverter()))
-        .apply(BigQueryIO.Write.to(tableSpec)
-            .withSchema(StringToRowConverter.getSchema()));
-
-    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.waitToFinish(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java
deleted file mode 100644
index 79ce823..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java
+++ /dev/null
@@ -1,432 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * An example that computes a basic TF-IDF search table for a directory or GCS prefix.
- *
- * <p>Concepts: joining data; side inputs; logging
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }</pre>
- * and a local output file or output prefix on GCS:
- * <pre>{@code
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * and an output prefix on GCS:
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
- *
- * <p>The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with
- * {@code --input}.
- */
-public class TfIdf {
-  /**
-   * Options supported by {@link TfIdf}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Path to the directory or GCS prefix containing files to read from")
-    @Default.String("gs://dataflow-samples/shakespeare/")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Prefix of output URI to write to")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  /**
-   * Lists documents contained beneath the {@code options.input} prefix/directory.
-   */
-  public static Set<URI> listInputDocuments(Options options)
-      throws URISyntaxException, IOException {
-    URI baseUri = new URI(options.getInput());
-
-    // List all documents in the directory or GCS prefix.
-    URI absoluteUri;
-    if (baseUri.getScheme() != null) {
-      absoluteUri = baseUri;
-    } else {
-      absoluteUri = new URI(
-          "file",
-          baseUri.getAuthority(),
-          baseUri.getPath(),
-          baseUri.getQuery(),
-          baseUri.getFragment());
-    }
-
-    Set<URI> uris = new HashSet<>();
-    if (absoluteUri.getScheme().equals("file")) {
-      File directory = new File(absoluteUri);
-      for (String entry : directory.list()) {
-        File path = new File(directory, entry);
-        uris.add(path.toURI());
-      }
-    } else if (absoluteUri.getScheme().equals("gs")) {
-      GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
-      URI gcsUriGlob = new URI(
-          absoluteUri.getScheme(),
-          absoluteUri.getAuthority(),
-          absoluteUri.getPath() + "*",
-          absoluteUri.getQuery(),
-          absoluteUri.getFragment());
-      for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
-        uris.add(entry.toUri());
-      }
-    }
-
-    return uris;
-  }
-
-  /**
-   * Reads the documents at the provided uris and returns all lines
-   * from the documents tagged with which document they are from.
-   */
-  public static class ReadDocuments
-      extends PTransform<PInput, PCollection<KV<URI, String>>> {
-    private Iterable<URI> uris;
-
-    public ReadDocuments(Iterable<URI> uris) {
-      this.uris = uris;
-    }
-
-    @Override
-    public Coder<?> getDefaultOutputCoder() {
-      return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
-    }
-
-    @Override
-    public PCollection<KV<URI, String>> apply(PInput input) {
-      Pipeline pipeline = input.getPipeline();
-
-      // Create one TextIO.Read transform for each document
-      // and add its output to a PCollectionList
-      PCollectionList<KV<URI, String>> urisToLines =
-          PCollectionList.empty(pipeline);
-
-      // TextIO.Read supports:
-      //  - file: URIs and paths locally
-      //  - gs: URIs on the service
-      for (final URI uri : uris) {
-        String uriString;
-        if (uri.getScheme().equals("file")) {
-          uriString = new File(uri).getPath();
-        } else {
-          uriString = uri.toString();
-        }
-
-        PCollection<KV<URI, String>> oneUriToLines = pipeline
-            .apply(TextIO.Read.from(uriString)
-                .named("TextIO.Read(" + uriString + ")"))
-            .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
-
-        urisToLines = urisToLines.and(oneUriToLines);
-      }
-
-      return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
-    }
-  }
-
-  /**
-   * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
-   * where the key is the document's URI and the value is a piece
-   * of the document's content. The output is mapping from terms to
-   * scores for each document URI.
-   */
-  public static class ComputeTfIdf
-      extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
-    public ComputeTfIdf() { }
-
-    @Override
-    public PCollection<KV<String, KV<URI, Double>>> apply(
-      PCollection<KV<URI, String>> uriToContent) {
-
-      // Compute the total number of documents, and
-      // prepare this singleton PCollectionView for
-      // use as a side input.
-      final PCollectionView<Long> totalDocuments =
-          uriToContent
-          .apply("GetURIs", Keys.<URI>create())
-          .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
-          .apply(Count.<URI>globally())
-          .apply(View.<Long>asSingleton());
-
-      // Create a collection of pairs mapping a URI to each
-      // of the words in the document associated with that that URI.
-      PCollection<KV<URI, String>> uriToWords = uriToContent
-          .apply(ParDo.named("SplitWords").of(
-              new DoFn<KV<URI, String>, KV<URI, String>>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  URI uri = c.element().getKey();
-                  String line = c.element().getValue();
-                  for (String word : line.split("\\W+")) {
-                    // Log INFO messages when the word “love” is found.
-                    if (word.toLowerCase().equals("love")) {
-                      LOG.info("Found {}", word.toLowerCase());
-                    }
-
-                    if (!word.isEmpty()) {
-                      c.output(KV.of(uri, word.toLowerCase()));
-                    }
-                  }
-                }
-              }));
-
-      // Compute a mapping from each word to the total
-      // number of documents in which it appears.
-      PCollection<KV<String, Long>> wordToDocCount = uriToWords
-          .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
-          .apply(Values.<String>create())
-          .apply("CountDocs", Count.<String>perElement());
-
-      // Compute a mapping from each URI to the total
-      // number of words in the document associated with that URI.
-      PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
-          .apply("GetURIs2", Keys.<URI>create())
-          .apply("CountWords", Count.<URI>perElement());
-
-      // Count, for each (URI, word) pair, the number of
-      // occurrences of that word in the document associated
-      // with the URI.
-      PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
-          .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
-
-      // Adjust the above collection to a mapping from
-      // (URI, word) pairs to counts into an isomorphic mapping
-      // from URI to (word, count) pairs, to prepare for a join
-      // by the URI key.
-      PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
-          .apply(ParDo.named("ShiftKeys").of(
-              new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  URI uri = c.element().getKey().getKey();
-                  String word = c.element().getKey().getValue();
-                  Long occurrences = c.element().getValue();
-                  c.output(KV.of(uri, KV.of(word, occurrences)));
-                }
-              }));
-
-      // Prepare to join the mapping of URI to (word, count) pairs with
-      // the mapping of URI to total word counts, by associating
-      // each of the input PCollection<KV<URI, ...>> with
-      // a tuple tag. Each input must have the same key type, URI
-      // in this case. The type parameter of the tuple tag matches
-      // the types of the values for each collection.
-      final TupleTag<Long> wordTotalsTag = new TupleTag<Long>();
-      final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<KV<String, Long>>();
-      KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
-          .of(wordTotalsTag, uriToWordTotal)
-          .and(wordCountsTag, uriToWordAndCount);
-
-      // Perform a CoGroupByKey (a sort of pre-join) on the prepared
-      // inputs. This yields a mapping from URI to a CoGbkResult
-      // (CoGroupByKey Result). The CoGbkResult is a mapping
-      // from the above tuple tags to the values in each input
-      // associated with a particular URI. In this case, each
-      // KV<URI, CoGbkResult> group a URI with the total number of
-      // words in that document as well as all the (word, count)
-      // pairs for particular words.
-      PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
-          .apply("CoGroupByUri", CoGroupByKey.<URI>create());
-
-      // Compute a mapping from each word to a (URI, term frequency)
-      // pair for each URI. A word's term frequency for a document
-      // is simply the number of times that word occurs in the document
-      // divided by the total number of words in the document.
-      PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
-          .apply(ParDo.named("ComputeTermFrequencies").of(
-              new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  URI uri = c.element().getKey();
-                  Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
-
-                  for (KV<String, Long> wordAndCount
-                           : c.element().getValue().getAll(wordCountsTag)) {
-                    String word = wordAndCount.getKey();
-                    Long wordCount = wordAndCount.getValue();
-                    Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
-                    c.output(KV.of(word, KV.of(uri, termFrequency)));
-                  }
-                }
-              }));
-
-      // Compute a mapping from each word to its document frequency.
-      // A word's document frequency in a corpus is the number of
-      // documents in which the word appears divided by the total
-      // number of documents in the corpus. Note how the total number of
-      // documents is passed as a side input; the same value is
-      // presented to each invocation of the DoFn.
-      PCollection<KV<String, Double>> wordToDf = wordToDocCount
-          .apply(ParDo
-              .named("ComputeDocFrequencies")
-              .withSideInputs(totalDocuments)
-              .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  String word = c.element().getKey();
-                  Long documentCount = c.element().getValue();
-                  Long documentTotal = c.sideInput(totalDocuments);
-                  Double documentFrequency = documentCount.doubleValue()
-                      / documentTotal.doubleValue();
-
-                  c.output(KV.of(word, documentFrequency));
-                }
-              }));
-
-      // Join the term frequency and document frequency
-      // collections, each keyed on the word.
-      final TupleTag<KV<URI, Double>> tfTag = new TupleTag<KV<URI, Double>>();
-      final TupleTag<Double> dfTag = new TupleTag<Double>();
-      PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
-          .of(tfTag, wordToUriAndTf)
-          .and(dfTag, wordToDf)
-          .apply(CoGroupByKey.<String>create());
-
-      // Compute a mapping from each word to a (URI, TF-IDF) score
-      // for each URI. There are a variety of definitions of TF-IDF
-      // ("term frequency - inverse document frequency") score;
-      // here we use a basic version that is the term frequency
-      // divided by the log of the document frequency.
-      PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = wordToUriAndTfAndDf
-          .apply(ParDo.named("ComputeTfIdf").of(
-              new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  String word = c.element().getKey();
-                  Double df = c.element().getValue().getOnly(dfTag);
-
-                  for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
-                    URI uri = uriAndTf.getKey();
-                    Double tf = uriAndTf.getValue();
-                    Double tfIdf = tf * Math.log(1 / df);
-                    c.output(KV.of(word, KV.of(uri, tfIdf)));
-                  }
-                }
-              }));
-
-      return wordToUriAndTfIdf;
-    }
-
-    // Instantiate Logger.
-    // It is suggested that the user specify the class name of the containing class
-    // (in this case ComputeTfIdf).
-    private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
-  }
-
-  /**
-   * A {@link PTransform} to write, in CSV format, a mapping from term and URI
-   * to score.
-   */
-  public static class WriteTfIdf
-      extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
-    private String output;
-
-    public WriteTfIdf(String output) {
-      this.output = output;
-    }
-
-    @Override
-    public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
-      return wordToUriAndTfIdf
-          .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
-            @Override
-            public void processElement(ProcessContext c) {
-              c.output(String.format("%s,\t%s,\t%f",
-                  c.element().getKey(),
-                  c.element().getValue().getKey(),
-                  c.element().getValue().getValue()));
-            }
-          }))
-          .apply(TextIO.Write
-              .to(output)
-              .withSuffix(".csv"));
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline pipeline = Pipeline.create(options);
-    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
-    pipeline
-        .apply(new ReadDocuments(listInputDocuments(options)))
-        .apply(new ComputeTfIdf())
-        .apply(new WriteTfIdf(options.getOutput()));
-
-    pipeline.run();
-  }
-}


[63/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
index 63a43b5..02a36cf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * Enforces that all elements in a {@link PCollection} can be encoded using that

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
index c32d820..9d8fc43 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 
 import java.util.Objects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
index 044402c..cfbf7b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import java.util.concurrent.ExecutorService;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
index a17292f..3463d08 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItem;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItems;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Optional;
 import com.google.common.cache.CacheBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
index e7ad80c..4e23dde 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Flatten.FlattenPCollectionList;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 
 /**
  * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
index 9c46a41..7833d42 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.TypedPValue;
 
 /**
  * A base class for implementing {@link PTransform} overrides, which behave identically to the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
index 4b478ad..ec0af8d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
@@ -15,35 +15,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.StepTransformResult.Builder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+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.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItemCoder;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.sdk.util.WindowedValue;
+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 org.apache.beam.sdk.values.POutput;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.StepTransformResult.Builder;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItem;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItemCoder;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItems;
-import com.google.cloud.dataflow.sdk.util.SystemReduceFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.common.annotations.VisibleForTesting;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
index 44670e8..0852269 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
@@ -15,23 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 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.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.api.client.util.Throwables;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.MutationDetector;
-import com.google.cloud.dataflow.sdk.util.MutationDetectors;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.SetMultimap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
index 8b7ccba..028870a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.MutationDetector;
-import com.google.cloud.dataflow.sdk.util.MutationDetectors;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
 
 import java.util.IdentityHashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
index 193d6a4..4725e27 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundle.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundle.java
index 2fb6536..d3da9e1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundle.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundle.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
index 89ec718..2d02401 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
index 6d2e557..4678a53 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessCreate.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessCreate.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessCreate.java
index 28b4de7..c29d5ce 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessCreate.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessCreate.java
@@ -15,23 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.OffsetBasedSource;
-import com.google.cloud.dataflow.sdk.io.OffsetBasedSource.OffsetBasedReader;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Create.Values;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.OffsetBasedSource;
+import org.apache.beam.sdk.io.OffsetBasedSource.OffsetBasedReader;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Create.Values;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
index 078827d..d439ba7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
@@ -15,32 +15,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
index 7d7ba39..1430c98 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.BaseExecutionContext;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.BaseExecutionContext;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.common.worker.StateSampler;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
 
 /**
  * Execution Context for the {@link InProcessPipelineRunner}.
@@ -56,7 +56,7 @@ class InProcessExecutionContext
    * Step Context for the {@link InProcessPipelineRunner}.
    */
   public class InProcessStepContext
-      extends com.google.cloud.dataflow.sdk.util.BaseExecutionContext.StepContext {
+      extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
     private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
     private InProcessTimerInternals timerInternals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
index 2792fd3..059aae6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
index d44ea78..bdc525a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.Hidden;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
index 797a533..a1a2567 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
@@ -15,43 +15,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
+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;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
+import org.apache.beam.sdk.runners.inprocess.InProcessCreate.InProcessCreateOverrideFactory;
+import org.apache.beam.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+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.View.CreatePCollectionView;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorPipelineExtractor;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessCreate.InProcessCreateOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -338,7 +338,7 @@ public class InProcessPipelineRunner
      *
      * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
      * exception. Future calls to {@link #getState()} will return
-     * {@link com.google.cloud.dataflow.sdk.PipelineResult.State#FAILED}.
+     * {@link org.apache.beam.sdk.PipelineResult.State#FAILED}.
      *
      * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
      * {@link PCollection}, and the {@link PipelineRunner} was created with

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
index e031aa6..8d29d01 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
 import com.google.common.collect.ImmutableList;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
index 70c2f20..6bf6e8a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.PCollectionViewWindow;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollectionView;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Throwables;
 import com.google.common.cache.CacheBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
index b2c651e..3422efd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimerInternals;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
index 41d6672..ed77f70 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
index 5360890..0e6b7e8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
 
 import java.util.HashSet;
 import java.util.Set;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
index de2a77a..4a3d17a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * Enforcement tools that verify that executing code conforms to the model.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
index 2bc01ce..1fa36d6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 
 /**
  * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
index 0ed3f58..71039fa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
index 77fa585..2b4bf09 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 interface PTransformOverrideFactory {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
index 4b4d699..a68fa53 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
index 1d6563d..2b95574 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
index 28c86a9..044b7e0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.Bound;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.Bound;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.Collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
index 50d0ed9..a90cd7b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
 
 class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
   public static <InputT> PassthroughTransformEvaluator<InputT> create(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
index fc6419e..88630ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PDone;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
index a6683fe..9c4d9aa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
 import com.google.common.base.MoreObjects;
 
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
index a1fd657..8874eda 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
index af433c4..fac5a40 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.TextIO.Write.Bound;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+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 {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
index 9828fc6..e002329 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
 
 /**
  * An evaluator of a specific application of a transform. Will be used for at least one

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
index c5d657a..a9f6759 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
index 113bad8..f6542b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.Window;
 
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Flatten.FlattenPCollectionList;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
 import com.google.common.collect.ImmutableMap;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
index 872aa80..3a7bedc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+
 import com.google.common.base.Throwables;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
index 67fe169..600102f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 /**
  * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
index 880ad7f..3194340 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import com.google.common.base.MoreObjects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
index c9d2929..0cebf43 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.io.Read.Unbounded;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.io.Read.Unbounded;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.io.IOException;
 import java.util.Queue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
index 112ea0c..0b54ba8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
index 230985d..3e4aca6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy;
 
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
 import com.google.common.collect.ComparisonChain;
 import com.google.common.collect.Ordering;
 



[69/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f9d4dcd..895ecef 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,18 +17,20 @@
  */
 package org.apache.beam.runners.flink.translation.types;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+
+import com.google.common.base.Preconditions;
+
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.AtomicType;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import com.google.common.base.Preconditions;
 
 /**
  * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s.
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s.
  */
 public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index 4e81054..c6f3921 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -19,8 +19,9 @@ package org.apache.beam.runners.flink.translation.types;
 
 import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
 import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -32,7 +33,7 @@ import java.io.ObjectInputStream;
 
 /**
  * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s
  */
 public class CoderTypeSerializer<T> extends TypeSerializer<T> {
   

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
index 3912295..6f0c651 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
@@ -18,9 +18,10 @@
 package org.apache.beam.runners.flink.translation.types;
 
 import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.values.KV;
+
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -31,7 +32,7 @@ import java.io.ObjectInputStream;
 
 /**
  * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link com.google.cloud.dataflow.sdk.coders.KvCoder}. We have a special comparator
+ * {@link org.apache.beam.sdk.coders.KvCoder}. We have a special comparator
  * for {@link KV} that always compares on the key only.
  */
 public class KvCoderComperator <K, V> extends TypeComparator<KV<K, V>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
index 8862d48..74f3821 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
@@ -17,20 +17,22 @@
  */
 package org.apache.beam.runners.flink.translation.types;
 
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.common.base.Preconditions;
+
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompositeType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import com.google.common.base.Preconditions;
 
 import java.util.List;
 
 /**
  * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.KvCoder}.
+ * Dataflow {@link org.apache.beam.sdk.coders.KvCoder}.
  */
 public class KvCoderTypeInformation<K, V> extends CompositeType<KV<K, V>> {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
index 8bc3620..7b48208 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
@@ -25,7 +25,7 @@ import java.io.IOException;
 
 /**
  * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
- * {@link com.google.cloud.dataflow.sdk.coders.VoidCoder}. We need this because Flink does not
+ * {@link org.apache.beam.sdk.coders.VoidCoder}. We need this because Flink does not
  * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead
  * that behaves like a {@code null}, hopefully.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
index 445d411..e5567d3 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
@@ -17,17 +17,19 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+
 import com.google.common.collect.Lists;
+
 import org.apache.flink.api.common.accumulators.Accumulator;
 
 import java.io.Serializable;
 
 /**
- * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
+ * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn}
  * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
- * the combine function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
+ * the combine function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo}
  * operation.
  */
 public class CombineFnAggregatorWrapper<AI, AA, AR> implements Aggregator<AI, AR>, Accumulator<AI, Serializable> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
index 6a3cf50..f1b8c73 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
@@ -26,7 +26,7 @@ import java.io.InputStream;
 /**
  * Wrapper for {@link DataInputView}. We need this because Flink reads data using a
  * {@link org.apache.flink.core.memory.DataInputView} while
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s expect an
  * {@link java.io.InputStream}.
  */
 public class DataInputViewWrapper extends InputStream {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
index 6bd2240..148f960 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
@@ -25,7 +25,7 @@ import java.io.OutputStream;
 /**
  * Wrapper for {@link org.apache.flink.core.memory.DataOutputView}. We need this because
  * Flink writes data using a {@link org.apache.flink.core.memory.DataInputView} while
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s expect an
  * {@link java.io.OutputStream}.
  */
 public class DataOutputViewWrapper extends OutputStream {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
index 4409586..eb32fa2 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
@@ -17,18 +17,20 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+
 import org.apache.flink.api.common.accumulators.Accumulator;
 
 import java.io.Serializable;
 
 /**
- * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
+ * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn}
  * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
- * the function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
+ * the function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo}
  * operation.
  */
 public class SerializableFnAggregatorWrapper<AI, AO> implements Aggregator<AI, AO>, Accumulator<AI, Serializable> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
index 4c2475d..45267b8 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
@@ -18,11 +18,14 @@
 
 package org.apache.beam.runners.flink.translation.wrappers;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.io.Sink;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.io.Sink;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Write;
+
 import com.google.common.base.Preconditions;
-import com.google.cloud.dataflow.sdk.transforms.Write;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.flink.api.common.io.OutputFormat;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.AbstractID;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
index cd5cd40..26e6297 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -17,11 +17,12 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers;
 
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.options.PipelineOptions;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.Configuration;
@@ -38,7 +39,7 @@ import java.util.List;
 
 /**
  * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a
- * Dataflow {@link com.google.cloud.dataflow.sdk.io.Source}.
+ * Dataflow {@link org.apache.beam.sdk.io.Source}.
  */
 public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>> {
   private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
index cde2b35..c3672c0 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
@@ -17,7 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers;
 
-import com.google.cloud.dataflow.sdk.io.Source;
+import org.apache.beam.sdk.io.Source;
+
 import org.apache.flink.core.io.InputSplit;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 10c8bbf..bb6ed67 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
@@ -18,18 +18,22 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+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 com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Throwables;
+
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -40,8 +44,8 @@ import org.joda.time.format.PeriodFormat;
 import java.util.Collection;
 
 /**
- * An abstract class that encapsulates the common code of the the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound}
- * and {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and
+ * An abstract class that encapsulates the common code of the the {@link org.apache.beam.sdk.transforms.ParDo.Bound}
+ * and {@link org.apache.beam.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and
  * {@link FlinkParDoBoundMultiWrapper} for the actual wrappers of the aforementioned transformations.
  * */
 public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 3dc5a79..8e7493e 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
@@ -19,21 +19,41 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.*;
-import com.google.cloud.dataflow.sdk.coders.*;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.values.*;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.AbstractFlinkTimerInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointReader;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointUtils;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointWriter;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+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.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;
+
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.core.memory.DataInputView;
@@ -41,20 +61,29 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.operators.*;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
 import org.joda.time.Instant;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * This class is the key class implementing all the windowing/triggering logic of Apache Beam.
  * To provide full compatibility and support for all the windowing/triggering combinations offered by
  * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in
- * ({@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}.
+ * ({@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}.
  * <p/>
  * In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
  * grouped by key</b>. Each of the elements that enter here, registers a timer
@@ -66,7 +95,7 @@ import java.util.*;
  * When a watermark arrives, all the registered timers are checked to see which ones are ready to
  * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from
  * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers}
- * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}
+ * list, and are fed into the {@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}
  * for furhter processing.
  */
 public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
@@ -108,11 +137,11 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
    * <p/>
    * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
    * is that this method assumes that a combiner function is provided
-   * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+   * (see {@link org.apache.beam.sdk.transforms.Combine.KeyedCombineFn}).
    * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
    *
    * @param options            the general job configuration options.
-   * @param input              the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+   * @param input              the input Dataflow {@link org.apache.beam.sdk.values.PCollection}.
    * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
    * @param combiner           the combiner to be used.
    * @param outputKvCoder      the type of the output values.
@@ -151,10 +180,10 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
    * <p/>
    * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
    * is that this method assumes no combiner function
-   * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+   * (see {@link org.apache.beam.sdk.transforms.Combine.KeyedCombineFn}).
    *
    * @param options            the general job configuration options.
-   * @param input              the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+   * @param input              the input Dataflow {@link org.apache.beam.sdk.values.PCollection}.
    * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
    */
   public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
@@ -224,9 +253,9 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
   }
 
   /**
-   * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn},
+   * Create the adequate {@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn},
    * <b> if not already created</b>.
-   * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
+   * If a {@link org.apache.beam.sdk.transforms.Combine.KeyedCombineFn} was provided, then
    * a function with that combiner is created, so that elements are combined as they arrive. This is
    * done for speed and (in most of the cases) for reduction of the per-window state.
    */
@@ -524,7 +553,7 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
       return new WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>>() {
 
         @Override
-        public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
+        public org.apache.beam.sdk.util.state.StateInternals stateInternals() {
           return stateInternals;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
index 1a6a665..3bf566b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -19,11 +19,12 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 df7f953..a30a544 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,21 +17,23 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.util.WindowedValue;
+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;
 
 import java.util.Map;
 
 /**
- * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} Beam transformation.
+ * A wrapper for the {@link org.apache.beam.sdk.transforms.ParDo.BoundMulti} Beam transformation.
  * */
 public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
index 2ed5620..4def0c6 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -17,23 +17,27 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Collection;
 
 /**
- * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} Beam transformation.
+ * A wrapper for the {@link org.apache.beam.sdk.transforms.ParDo.Bound} Beam transformation.
  * */
 public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
index f6c243f..d6aff7d 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -18,12 +18,12 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
 import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 82984cb..05a8c7a 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,16 +17,18 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+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.RichParallelSourceFunction;
+
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 
-import javax.annotation.Nullable;
 import java.util.List;
 
+import javax.annotation.Nullable;
+
 /**
  * A wrapper translating Flink Sources implementing the {@link SourceFunction} interface, into
  * unbounded Beam sources (see {@link UnboundedSource}).

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
index 1389e9d..08bdb50 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -17,15 +17,17 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+
 import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -36,7 +38,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.NoSuchElementException;
 
-import static com.google.common.base.Preconditions.checkArgument;
+import javax.annotation.Nullable;
 
 /**
  * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 43cf4b2..dcc7967 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -17,14 +17,16 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
+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.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.operators.StreamSource;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
@@ -38,7 +40,7 @@ import java.io.ObjectOutputStream;
 
 /**
  * A wrapper for Beam's unbounded sources. This class wraps around a source implementing the
- * {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded}  interface.
+ * {@link org.apache.beam.sdk.io.Read.Unbounded}  interface.
  *
  * For now we support non-parallel sources, checkpointing is WIP.
  * */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
index fc75948..9e55002 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -17,20 +17,19 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimerInternals;
+
 import org.joda.time.Instant;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.Serializable;
 
+import javax.annotation.Nullable;
+
 /**
  * An implementation of Beam's {@link TimerInternals}, that also provides serialization functionality.
  * The latter is used when snapshots of the current state are taken, for fault-tolerance.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 6cf46e5..352c550 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,22 +17,39 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.state.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.CombineWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTable;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+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;
 import org.joda.time.Instant;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
 
 /**
  * An implementation of the Beam {@link StateInternals}. This implementation simply keeps elements in memory.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
index 5aadccd..5a843ab 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
@@ -18,7 +18,9 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+
 import com.google.protobuf.ByteString;
+
 import org.apache.flink.core.memory.DataInputView;
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
index b2dc33c..4fbd6f0 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
@@ -18,14 +18,14 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+
 import org.joda.time.Instant;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
index 18e118a..d09157c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
@@ -18,7 +18,9 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+
 import com.google.protobuf.ByteString;
+
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
index 3536f87..113fee0 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
@@ -18,14 +18,16 @@
 
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
index 5ae0e83..ac0a3d7 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
@@ -17,13 +17,15 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 public class FlattenizeITCase extends JavaProgramTestBase {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
index 45ca830..bd149c7 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
@@ -18,12 +18,13 @@
 
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import org.junit.Test;
-
 import static org.junit.Assert.assertEquals;
 
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+import org.junit.Test;
+
 /**
  * Tests the proper registration of the Flink runner.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
index 86a0cd0..f015a66 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
 
 /**
- * {@link com.google.cloud.dataflow.sdk.Pipeline} for testing Dataflow programs on the
+ * {@link org.apache.beam.sdk.Pipeline} for testing Dataflow programs on the
  * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
  */
 public class FlinkTestPipeline extends Pipeline {
@@ -31,7 +31,7 @@ public class FlinkTestPipeline extends Pipeline {
   /**
    * Creates and returns a new test pipeline for batch execution.
    *
-   * <p> Use {@link com.google.cloud.dataflow.sdk.testing.PAssert} to add tests, then call
+   * <p> Use {@link org.apache.beam.sdk.testing.PAssert} to add tests, then call
    * {@link Pipeline#run} to execute the pipeline and check the tests.
    */
   public static FlinkTestPipeline createForBatch() {
@@ -41,7 +41,7 @@ public class FlinkTestPipeline extends Pipeline {
   /**
    * Creates and returns a new test pipeline for streaming execution.
    *
-   * <p> Use {@link com.google.cloud.dataflow.sdk.testing.PAssert} to add tests, then call
+   * <p> Use {@link org.apache.beam.sdk.testing.PAssert} to add tests, then call
    * {@link Pipeline#run} to execute the pipeline and check the tests.
    *
    * @return The Test Pipeline
@@ -53,7 +53,7 @@ public class FlinkTestPipeline extends Pipeline {
   /**
    * Creates and returns a new test pipeline for streaming or batch execution.
    *
-   * <p> Use {@link com.google.cloud.dataflow.sdk.testing.PAssert} to add tests, then call
+   * <p> Use {@link org.apache.beam.sdk.testing.PAssert} to add tests, then call
    * {@link Pipeline#run} to execute the pipeline and check the tests.
    *
    * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
index f60056d..47685b6 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
@@ -18,13 +18,14 @@
 package org.apache.beam.runners.flink;
 
 import org.apache.beam.runners.flink.util.JoinExamples;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
index 199602c..4d66fa4 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
index 403de29..a2ef4e2 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
@@ -17,16 +17,18 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
index 323c41b..bcad6f1 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
@@ -17,18 +17,20 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
index 524554a..471d326 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
@@ -17,13 +17,15 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+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.transforms.Create;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
index 54e92aa..0544f20 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
@@ -17,13 +17,15 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+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.transforms.Create;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
index 7f73b83..2c7c65e 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
@@ -17,13 +17,14 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollectionView;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
index b0fb7b8..547f3c3 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
@@ -17,16 +17,18 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.examples.complete.TfIdf;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.examples.complete.TfIdf;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.net.URI;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
index 2677c9e..3254e78 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
@@ -17,14 +17,16 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.examples.WordCount;
+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.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
index e73c456..6570e7d 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
@@ -17,19 +17,21 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+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.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
index 6b57d77..60dc74a 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
@@ -17,19 +17,21 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+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.Joiner;
+
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
index dfa15ce..71514d3 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
@@ -18,14 +18,18 @@
 
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.Sink;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Write;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.beam.sdk.Pipeline;
+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.transforms.Create;
+import org.apache.beam.sdk.transforms.Write;
+
 import com.google.common.base.Joiner;
+
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.test.util.JavaProgramTestBase;
@@ -34,8 +38,6 @@ import java.io.File;
 import java.io.PrintWriter;
 import java.net.URI;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests the translation of custom Write.Bound sinks.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 880da59..f3ceba7 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
@@ -19,18 +19,27 @@ package org.apache.beam.runners.flink.streaming;
 
 import org.apache.beam.runners.flink.FlinkTestPipeline;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.base.Throwables;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
@@ -73,9 +82,9 @@ public class GroupAlsoByWindowTest {
 
   /**
    * The default accumulation mode is
-   * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
+   * {@link org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
    * This strategy changes it to
-   * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
+   * {@link org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
    */
   private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
       fixedWindowWithCompoundTriggerStrategy

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
index 63e0bcf..9f7bd90 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -18,19 +18,20 @@
 package org.apache.beam.runners.flink.streaming;
 
 import org.apache.beam.runners.flink.FlinkTestPipeline;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+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.ParDo;
+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.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
-import org.apache.flink.api.java.tuple.Tuple2;
+
 import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.joda.time.Duration;
 import org.joda.time.Instant;


[19/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/package-info.java
deleted file mode 100644
index 8242fad..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-/**
- * Defines a {@link com.google.cloud.dataflow.sdk.coders.Coder}
- * for Protocol Buffers messages, {@code ProtoCoder}.
- *
- * @see com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder
- */
-package com.google.cloud.dataflow.sdk.coders.protobuf;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java
deleted file mode 100644
index 145804d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java
+++ /dev/null
@@ -1,811 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.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;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.reflect.ReflectData;
-
-import java.io.IOException;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.regex.Pattern;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link PTransform}s for reading and writing Avro files.
- *
- * <p>To read a {@link PCollection} from one or more Avro files, use
- * {@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.
- *
- * <p>It is required to specify {@link AvroIO.Read#withSchema}. To
- * read specific records, such as Avro-generated classes, provide an
- * Avro-generated class type. To read {@link GenericRecord GenericRecords}, provide either
- * a {@link Schema} object or an Avro schema in a JSON-encoded string form.
- * An exception will be thrown if a record doesn't match the specified
- * schema.
- *
- * <p>For example:
- * <pre> {@code
- * Pipeline p = ...;
- *
- * // 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));
- *
- * // 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));
- * } </pre>
- *
- * <p>To write a {@link PCollection} to one or more Avro files, use
- * {@link AvroIO.Write}, specifying {@link AvroIO.Write#to} to specify
- * 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>"}), and optionally
- * {@link AvroIO.Write#named} to specify the name of the pipeline step.
- *
- * <p>It is required to specify {@link AvroIO.Write#withSchema}. To
- * write specific records, such as Avro-generated classes, provide an
- * Avro-generated class type. To write {@link GenericRecord GenericRecords}, provide either
- * a {@link Schema} object or a schema in a JSON-encoded string form.
- * An exception will be thrown if a record doesn't match the specified
- * schema.
- *
- * <p>For example:
- * <pre> {@code
- * // A simple Write to a local file (only runs locally):
- * PCollection<AvroAutoGenClass> records = ...;
- * records.apply(AvroIO.Write.to("/path/to/file.avro")
- *                           .withSchema(AvroAutoGenClass.class));
- *
- * // A Write to a sharded GCS file (runs locally and via the Google Cloud
- * // 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"));
- * } </pre>
- *
- * <p><h3>Permissions</h3>
- * Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding {@link PipelineRunner}s for
- * more details.
- */
-public class AvroIO {
-  /**
-   * A root {@link PTransform} that reads from an Avro file (or multiple Avro
-   * files matching a pattern) and returns a {@link PCollection} containing
-   * 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)
-     * with the given name or pattern. This can be 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>"} (if running locally or via
-     * the Google Cloud Dataflow service). Standard
-     * <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html">Java
-     * Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
-     */
-    public static Bound<GenericRecord> from(String filepattern) {
-      return new Bound<>(GenericRecord.class).from(filepattern);
-    }
-
-    /**
-     * Returns a {@link PTransform} that reads Avro file(s)
-     * containing records whose type is the specified Avro-generated class.
-     *
-     * @param <T> the type of the decoded elements, and the elements
-     * of the resulting {@link PCollection}
-     */
-    public static <T> Bound<T> withSchema(Class<T> type) {
-      return new Bound<>(type).withSchema(type);
-    }
-
-    /**
-     * Returns a {@link PTransform} that reads Avro file(s)
-     * containing records of the specified schema.
-     */
-    public static Bound<GenericRecord> withSchema(Schema schema) {
-      return new Bound<>(GenericRecord.class).withSchema(schema);
-    }
-
-    /**
-     * Returns a {@link PTransform} that reads Avro file(s)
-     * containing records of the specified schema in a JSON-encoded
-     * string form.
-     */
-    public static Bound<GenericRecord> withSchema(String schema) {
-      return withSchema((new Schema.Parser()).parse(schema));
-    }
-
-    /**
-     * Returns a {@link PTransform} that reads Avro file(s)
-     * that has GCS path validation on pipeline creation disabled.
-     *
-     * <p>This can be useful in the case where the GCS input location does
-     * not exist at the pipeline creation time, but is expected to be available
-     * at execution time.
-     */
-    public static Bound<GenericRecord> withoutValidation() {
-      return new Bound<>(GenericRecord.class).withoutValidation();
-    }
-
-    /**
-     * A {@link PTransform} that reads from an Avro file (or multiple Avro
-     * files matching a pattern) and returns a bounded {@link PCollection} containing
-     * the decoding of each record.
-     *
-     * @param <T> the type of each of the elements of the resulting
-     * PCollection
-     */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
-      /** The filepattern to read from. */
-      @Nullable
-      final String filepattern;
-      /** The class type of the records. */
-      final Class<T> type;
-      /** The schema of the input file. */
-      @Nullable
-      final Schema schema;
-      /** An option to indicate if input validation is desired. Default is true. */
-      final boolean validate;
-
-      Bound(Class<T> type) {
-        this(null, null, type, null, true);
-      }
-
-      Bound(String name, String filepattern, Class<T> type, Schema schema, boolean validate) {
-        super(name);
-        this.filepattern = filepattern;
-        this.type = type;
-        this.schema = schema;
-        this.validate = validate;
-      }
-
-      /**
-       * 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.)
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> from(String filepattern) {
-        return new Bound<>(name, filepattern, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that reads Avro file(s) containing records whose type is the
-       * specified Avro-generated class.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param <X> the type of the decoded elements and the elements of
-       * the resulting PCollection
-       */
-      public <X> Bound<X> withSchema(Class<X> type) {
-        return new Bound<>(name, filepattern, type, ReflectData.get().getSchema(type), validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that reads Avro file(s) containing records of the specified schema.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<GenericRecord> withSchema(Schema schema) {
-        return new Bound<>(name, filepattern, GenericRecord.class, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that reads Avro file(s) containing records of the specified schema
-       * in a JSON-encoded string form.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<GenericRecord> withSchema(String schema) {
-        return withSchema((new Schema.Parser()).parse(schema));
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that has GCS input path validation on pipeline creation disabled.
-       *
-       * <p>Does not modify this object.
-       *
-       * <p>This can be useful in the case where the GCS input location does
-       * not exist at the pipeline creation time, but is expected to be
-       * available at execution time.
-       */
-      public Bound<T> withoutValidation() {
-        return new Bound<>(name, filepattern, type, schema, false);
-      }
-
-      @Override
-      public PCollection<T> apply(PInput input) {
-        if (filepattern == null) {
-          throw new IllegalStateException(
-              "need to set the filepattern of an AvroIO.Read transform");
-        }
-        if (schema == null) {
-          throw new IllegalStateException("need to set the schema of an AvroIO.Read transform");
-        }
-        if (validate) {
-          try {
-            checkState(
-                !IOChannelUtils.getFactory(filepattern).match(filepattern).isEmpty(),
-                "Unable to find any files matching %s",
-                filepattern);
-          } catch (IOException e) {
-            throw new IllegalStateException(
-                String.format("Failed to validate %s", filepattern), e);
-          }
-        }
-
-        @SuppressWarnings("unchecked")
-        Bounded<T> read =
-            type == GenericRecord.class
-                ? (Bounded<T>) com.google.cloud.dataflow.sdk.io.Read.from(
-                    AvroSource.from(filepattern).withSchema(schema))
-                : com.google.cloud.dataflow.sdk.io.Read.from(
-                    AvroSource.from(filepattern).withSchema(type));
-
-        PCollection<T> pcol = input.getPipeline().apply("Read", read);
-        // Honor the default output coder that would have been used by this PTransform.
-        pcol.setCoder(getDefaultOutputCoder());
-        return pcol;
-      }
-
-      @Override
-      protected Coder<T> getDefaultOutputCoder() {
-        return AvroCoder.of(type, schema);
-      }
-
-      public String getFilepattern() {
-        return filepattern;
-      }
-
-      public Schema getSchema() {
-        return schema;
-      }
-
-      public boolean needsValidation() {
-        return validate;
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Read() {}
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * A root {@link PTransform} that writes a {@link PCollection} to an Avro file (or
-   * 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)
-     * with the given prefix. This can be a local filename
-     * (if running locally), or a Google Cloud Storage filename of
-     * the form {@code "gs://<bucket>/<filepath>"}
-     * (if running locally or via the Google Cloud Dataflow service).
-     *
-     * <p>The files written will begin with this prefix, followed by
-     * a shard identifier (see {@link Bound#withNumShards}, and end
-     * in a common extension, if given by {@link Bound#withSuffix}.
-     */
-    public static Bound<GenericRecord> to(String prefix) {
-      return new Bound<>(GenericRecord.class).to(prefix);
-    }
-
-    /**
-     * Returns a {@link PTransform} that writes to the file(s) with the
-     * given filename suffix.
-     */
-    public static Bound<GenericRecord> withSuffix(String filenameSuffix) {
-      return new Bound<>(GenericRecord.class).withSuffix(filenameSuffix);
-    }
-
-    /**
-     * Returns a {@link PTransform} that uses the provided shard count.
-     *
-     * <p>Constraining the number of shards is likely to reduce
-     * the performance of a pipeline. Setting this value is not recommended
-     * unless you require a specific number of output files.
-     *
-     * @param numShards the number of shards to use, or 0 to let the system
-     *                  decide.
-     */
-    public static Bound<GenericRecord> withNumShards(int numShards) {
-      return new Bound<>(GenericRecord.class).withNumShards(numShards);
-    }
-
-    /**
-     * Returns a {@link PTransform} that uses the given shard name
-     * template.
-     *
-     * <p>See {@link ShardNameTemplate} for a description of shard templates.
-     */
-    public static Bound<GenericRecord> withShardNameTemplate(String shardTemplate) {
-      return new Bound<>(GenericRecord.class).withShardNameTemplate(shardTemplate);
-    }
-
-    /**
-     * Returns a {@link PTransform} that forces a single file as
-     * output.
-     *
-     * <p>Constraining the number of shards is likely to reduce
-     * the performance of a pipeline. Setting this value is not recommended
-     * unless you require a specific number of output files.
-     */
-    public static Bound<GenericRecord> withoutSharding() {
-      return new Bound<>(GenericRecord.class).withoutSharding();
-    }
-
-    /**
-     * Returns a {@link PTransform} that writes Avro file(s)
-     * containing records whose type is the specified Avro-generated class.
-     *
-     * @param <T> the type of the elements of the input PCollection
-     */
-    public static <T> Bound<T> withSchema(Class<T> type) {
-      return new Bound<>(type).withSchema(type);
-    }
-
-    /**
-     * Returns a {@link PTransform} that writes Avro file(s)
-     * containing records of the specified schema.
-     */
-    public static Bound<GenericRecord> withSchema(Schema schema) {
-      return new Bound<>(GenericRecord.class).withSchema(schema);
-    }
-
-    /**
-     * Returns a {@link PTransform} that writes Avro file(s)
-     * containing records of the specified schema in a JSON-encoded
-     * string form.
-     */
-    public static Bound<GenericRecord> withSchema(String schema) {
-      return withSchema((new Schema.Parser()).parse(schema));
-    }
-
-    /**
-     * Returns a {@link PTransform} that writes Avro file(s) that has GCS path validation on
-     * pipeline creation disabled.
-     *
-     * <p>This can be useful in the case where the GCS output location does
-     * not exist at the pipeline creation time, but is expected to be available
-     * at execution time.
-     */
-    public static Bound<GenericRecord> withoutValidation() {
-      return new Bound<>(GenericRecord.class).withoutValidation();
-    }
-
-    /**
-     * A {@link PTransform} that writes a bounded {@link PCollection} to an Avro file (or
-     * multiple Avro files matching a sharding pattern).
-     *
-     * @param <T> the type of each of the elements of the input PCollection
-     */
-    public static class Bound<T> extends PTransform<PCollection<T>, PDone> {
-      /** The filename to write to. */
-      @Nullable
-      final String filenamePrefix;
-      /** Suffix to use for each filename. */
-      final String filenameSuffix;
-      /** Requested number of shards. 0 for automatic. */
-      final int numShards;
-      /** Shard template string. */
-      final String shardTemplate;
-      /** The class type of the records. */
-      final Class<T> type;
-      /** The schema of the output file. */
-      @Nullable
-      final Schema schema;
-      /** An option to indicate if output validation is desired. Default is true. */
-      final boolean validate;
-
-      Bound(Class<T> type) {
-        this(null, null, "", 0, ShardNameTemplate.INDEX_OF_MAX, type, null, true);
-      }
-
-      Bound(
-          String name,
-          String filenamePrefix,
-          String filenameSuffix,
-          int numShards,
-          String shardTemplate,
-          Class<T> type,
-          Schema schema,
-          boolean validate) {
-        super(name);
-        this.filenamePrefix = filenamePrefix;
-        this.filenameSuffix = filenameSuffix;
-        this.numShards = numShards;
-        this.shardTemplate = shardTemplate;
-        this.type = type;
-        this.schema = schema;
-        this.validate = validate;
-      }
-
-      /**
-       * 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
-       * about filenames.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> to(String filenamePrefix) {
-        validateOutputComponent(filenamePrefix);
-        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 suffix.
-       *
-       * <p>See {@link ShardNameTemplate} for a description of shard templates.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withSuffix(String filenameSuffix) {
-        validateOutputComponent(filenameSuffix);
-        return new Bound<>(
-            name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that uses the provided shard count.
-       *
-       * <p>Constraining the number of shards is likely to reduce
-       * the performance of a pipeline. Setting this value is not recommended
-       * unless you require a specific number of output files.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param numShards the number of shards to use, or 0 to let the system
-       *                  decide.
-       * @see ShardNameTemplate
-       */
-      public Bound<T> withNumShards(int numShards) {
-        Preconditions.checkArgument(numShards >= 0);
-        return new Bound<>(
-            name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that uses the given shard name template.
-       *
-       * <p>Does not modify this object.
-       *
-       * @see ShardNameTemplate
-       */
-      public Bound<T> withShardNameTemplate(String shardTemplate) {
-        return new Bound<>(
-            name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that forces a single file as output.
-       *
-       * <p>This is a shortcut for
-       * {@code .withNumShards(1).withShardNameTemplate("")}
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withoutSharding() {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, 1, "", type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that writes to Avro file(s) containing records whose type is the
-       * specified Avro-generated class.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param <X> the type of the elements of the input PCollection
-       */
-      public <X> Bound<X> withSchema(Class<X> type) {
-        return new Bound<>(
-            name,
-            filenamePrefix,
-            filenameSuffix,
-            numShards,
-            shardTemplate,
-            type,
-            ReflectData.get().getSchema(type),
-            validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that writes to Avro file(s) containing records of the specified
-       * schema.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<GenericRecord> withSchema(Schema schema) {
-        return new Bound<>(
-            name,
-            filenamePrefix,
-            filenameSuffix,
-            numShards,
-            shardTemplate,
-            GenericRecord.class,
-            schema,
-            validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that writes to Avro file(s) containing records of the specified
-       * schema in a JSON-encoded string form.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<GenericRecord> withSchema(String schema) {
-        return withSchema((new Schema.Parser()).parse(schema));
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
-       * that has GCS output path validation on pipeline creation disabled.
-       *
-       * <p>Does not modify this object.
-       *
-       * <p>This can be useful in the case where the GCS output location does
-       * not exist at the pipeline creation time, but is expected to be
-       * available at execution time.
-       */
-      public Bound<T> withoutValidation() {
-        return new Bound<>(
-            name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, false);
-      }
-
-      @Override
-      public PDone apply(PCollection<T> input) {
-        if (filenamePrefix == null) {
-          throw new IllegalStateException(
-              "need to set the filename prefix of an AvroIO.Write transform");
-        }
-        if (schema == null) {
-          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",
-            com.google.cloud.dataflow.sdk.io.Write.to(
-                new AvroSink<>(
-                    filenamePrefix, filenameSuffix, shardTemplate, AvroCoder.of(type, schema))));
-      }
-
-      /**
-       * Returns the current shard name template string.
-       */
-      public String getShardNameTemplate() {
-        return shardTemplate;
-      }
-
-      @Override
-      protected Coder<Void> getDefaultOutputCoder() {
-        return VoidCoder.of();
-      }
-
-      public String getFilenamePrefix() {
-        return filenamePrefix;
-      }
-
-      public String getShardTemplate() {
-        return shardTemplate;
-      }
-
-      public int getNumShards() {
-        return numShards;
-      }
-
-      public String getFilenameSuffix() {
-        return filenameSuffix;
-      }
-
-      public Class<T> getType() {
-        return type;
-      }
-
-      public Schema getSchema() {
-        return schema;
-      }
-
-      public boolean needsValidation() {
-        return validate;
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Write() {}
-  }
-
-  // Pattern which matches old-style shard output patterns, which are now
-  // disallowed.
-  private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)");
-
-  private static void validateOutputComponent(String partialFilePattern) {
-    Preconditions.checkArgument(
-        !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(),
-        "Output name components are not allowed to contain @* or @N patterns: "
-        + partialFilePattern);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Disallow construction of utility class. */
-  private AvroIO() {}
-
-  /**
-   * A {@link FileBasedSink} for Avro files.
-   */
-  @VisibleForTesting
-  static class AvroSink<T> extends FileBasedSink<T> {
-    private final AvroCoder<T> coder;
-
-    @VisibleForTesting
-    AvroSink(
-        String baseOutputFilename, String extension, String fileNameTemplate, AvroCoder<T> coder) {
-      super(baseOutputFilename, extension, fileNameTemplate);
-      this.coder = coder;
-    }
-
-    @Override
-    public FileBasedSink.FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options) {
-      return new AvroWriteOperation<>(this, coder);
-    }
-
-    /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation
-     * FileBasedWriteOperation} for Avro files.
-     */
-    private static class AvroWriteOperation<T> extends FileBasedWriteOperation<T> {
-      private final AvroCoder<T> coder;
-
-      private AvroWriteOperation(AvroSink<T> sink, AvroCoder<T> coder) {
-        super(sink);
-        this.coder = coder;
-      }
-
-      @Override
-      public FileBasedWriter<T> createWriter(PipelineOptions options) throws Exception {
-        return new AvroWriter<>(this, coder);
-      }
-    }
-
-    /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
-     * for Avro files.
-     */
-    private static class AvroWriter<T> extends FileBasedWriter<T> {
-      private final AvroCoder<T> coder;
-      private DataFileWriter<T> dataFileWriter;
-
-      public AvroWriter(FileBasedWriteOperation<T> writeOperation, AvroCoder<T> coder) {
-        super(writeOperation);
-        this.mimeType = MimeTypes.BINARY;
-        this.coder = coder;
-      }
-
-      @SuppressWarnings("deprecation") // uses internal test functionality.
-      @Override
-      protected void prepareWrite(WritableByteChannel channel) throws Exception {
-        dataFileWriter = new DataFileWriter<>(coder.createDatumWriter());
-        dataFileWriter.create(coder.getSchema(), Channels.newOutputStream(channel));
-      }
-
-      @Override
-      public void write(T value) throws Exception {
-        dataFileWriter.append(value);
-      }
-
-      @Override
-      protected void writeFooter() throws Exception {
-        dataFileWriter.flush();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java
deleted file mode 100644
index 2e017c0..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java
+++ /dev/null
@@ -1,648 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.util.AvroUtils;
-import com.google.cloud.dataflow.sdk.util.AvroUtils.AvroMetadata;
-import com.google.cloud.dataflow.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;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
-import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream;
-import org.apache.commons.compress.compressors.xz.XZCompressorInputStream;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PushbackInputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.util.Collection;
-import java.util.zip.Inflater;
-import java.util.zip.InflaterInputStream;
-
-// CHECKSTYLE.OFF: JavadocStyle
-/**
- * A {@link FileBasedSource} for reading Avro files.
- *
- * <p>To read a {@link PCollection} of objects from one or more Avro files, use
- * {@link AvroSource#from} to specify the path(s) of the files to read. The {@link AvroSource} that
- * is returned will read objects of type {@link GenericRecord} with the schema(s) that were written
- * at file creation. To further configure the {@link AvroSource} to read with a user-defined schema,
- * or to return records of a type other than {@link GenericRecord}, use
- * {@link AvroSource#withSchema(Schema)} (using an Avro {@link Schema}),
- * {@link AvroSource#withSchema(String)} (using a JSON schema), or
- * {@link AvroSource#withSchema(Class)} (to return objects of the Avro-generated class specified).
- *
- * <p>An {@link AvroSource} can be read from using the {@link Read} transform. For example:
- *
- * <pre>
- * {@code
- * AvroSource<MyType> source = AvroSource.from(file.toPath()).withSchema(MyType.class);
- * PCollection<MyType> records = Read.from(mySource);
- * }
- * </pre>
- *
- * <p>The {@link AvroSource#readFromFileWithClass(String, Class)} method is a convenience method
- * that returns a read transform. For example:
- *
- * <pre>
- * {@code
- * PCollection<MyType> records = AvroSource.readFromFileWithClass(file.toPath(), MyType.class));
- * }
- * </pre>
- *
- * <p>This class's implementation is based on the <a
- * href="https://avro.apache.org/docs/1.7.7/spec.html">Avro 1.7.7</a> specification and implements
- * parsing of some parts of Avro Object Container Files. The rationale for doing so is that the Avro
- * API does not provide efficient ways of computing the precise offsets of blocks within a file,
- * which is necessary to support dynamic work rebalancing. However, whenever it is possible to use
- * the Avro API in a way that supports maintaining precise offsets, this class uses the Avro API.
- *
- * <p>Avro Object Container files store records in blocks. Each block contains a collection of
- * records. Blocks may be encoded (e.g., with bzip2, deflate, snappy, etc.). Blocks are delineated
- * from one another by a 16-byte sync marker.
- *
- * <p>An {@link AvroSource} for a subrange of a single file contains records in the blocks such that
- * the start offset of the block is greater than or equal to the start offset of the source and less
- * than the end offset of the source.
- *
- * <p>To use XZ-encoded Avro files, please include an explicit dependency on {@code xz-1.5.jar},
- * which has been marked as optional in the Maven {@code sdk/pom.xml} for Google Cloud Dataflow:
- *
- * <pre>{@code
- * <dependency>
- *   <groupId>org.tukaani</groupId>
- *   <artifactId>xz</artifactId>
- *   <version>1.5</version>
- * </dependency>
- * }</pre>
- *
- * <h3>Permissions</h3>
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding {@link PipelineRunner}s for
- * more details.
- *
- * @param <T> The type of records to be read from the source.
- */
-// CHECKSTYLE.ON: JavadocStyle
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public class AvroSource<T> extends BlockBasedSource<T> {
-  // Default minimum bundle size (chosen as two default-size Avro blocks to attempt to
-  // ensure that every source has at least one block of records).
-  // The default sync interval is 64k.
-  static final long DEFAULT_MIN_BUNDLE_SIZE = 2 * DataFileConstants.DEFAULT_SYNC_INTERVAL;
-
-  // The JSON schema used to encode records.
-  private final String readSchemaString;
-
-  // The JSON schema that was used to write the source Avro file (may differ from the schema we will
-  // use to read from it).
-  private final String fileSchemaString;
-
-  // The type of the records contained in the file.
-  private final Class<T> type;
-
-  // The following metadata fields are not user-configurable. They are extracted from the object
-  // container file header upon subsource creation.
-
-  // The codec used to encode the blocks in the Avro file. String value drawn from those in
-  // https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html
-  private final String codec;
-
-  // The object container file's 16-byte sync marker.
-  private final byte[] syncMarker;
-
-  // Default output coder, lazily initialized.
-  private transient AvroCoder<T> coder = null;
-
-  // Schema of the file, lazily initialized.
-  private transient Schema fileSchema;
-
-  // Schema used to encode records, lazily initialized.
-  private transient Schema readSchema;
-
-  /**
-   * Creates a {@link Read} transform that will read from an {@link AvroSource} that is configured
-   * to read records of the given type from a file pattern.
-   */
-  public static <T> Read.Bounded<T> readFromFileWithClass(String filePattern, Class<T> clazz) {
-    return Read.from(new AvroSource<T>(filePattern, DEFAULT_MIN_BUNDLE_SIZE,
-        ReflectData.get().getSchema(clazz).toString(), clazz, null, null));
-  }
-
-  /**
-   * Creates an {@link AvroSource} that reads from the given file name or pattern ("glob"). The
-   * returned source can be further configured by calling {@link #withSchema} to return a type other
-   * than {@link GenericRecord}.
-   */
-  public static AvroSource<GenericRecord> from(String fileNameOrPattern) {
-    return new AvroSource<>(
-        fileNameOrPattern, DEFAULT_MIN_BUNDLE_SIZE, null, GenericRecord.class, null, null);
-  }
-
-  /**
-   * Returns an {@link AvroSource} that's like this one but reads files containing records that
-   * conform to the given schema.
-   *
-   * <p>Does not modify this object.
-   */
-  public AvroSource<GenericRecord> withSchema(String schema) {
-    return new AvroSource<>(
-        getFileOrPatternSpec(), getMinBundleSize(), schema, GenericRecord.class, codec, syncMarker);
-  }
-
-  /**
-   * Returns an {@link AvroSource} that's like this one but reads files containing records that
-   * conform to the given schema.
-   *
-   * <p>Does not modify this object.
-   */
-  public AvroSource<GenericRecord> withSchema(Schema schema) {
-    return new AvroSource<>(getFileOrPatternSpec(), getMinBundleSize(), schema.toString(),
-        GenericRecord.class, codec, syncMarker);
-  }
-
-  /**
-   * Returns an {@link AvroSource} that's like this one but reads files containing records of the
-   * type of the given class.
-   *
-   * <p>Does not modify this object.
-   */
-  public <X> AvroSource<X> withSchema(Class<X> clazz) {
-    return new AvroSource<X>(getFileOrPatternSpec(), getMinBundleSize(),
-        ReflectData.get().getSchema(clazz).toString(), clazz, codec, syncMarker);
-  }
-
-  /**
-   * Returns an {@link AvroSource} that's like this one but uses the supplied minimum bundle size.
-   * Refer to {@link OffsetBasedSource} for a description of {@code minBundleSize} and its use.
-   *
-   * <p>Does not modify this object.
-   */
-  public AvroSource<T> withMinBundleSize(long minBundleSize) {
-    return new AvroSource<T>(
-        getFileOrPatternSpec(), minBundleSize, readSchemaString, type, codec, syncMarker);
-  }
-
-  private AvroSource(String fileNameOrPattern, long minBundleSize, String schema, Class<T> type,
-      String codec, byte[] syncMarker) {
-    super(fileNameOrPattern, minBundleSize);
-    this.readSchemaString = schema;
-    this.codec = codec;
-    this.syncMarker = syncMarker;
-    this.type = type;
-    this.fileSchemaString = null;
-  }
-
-  private AvroSource(String fileName, long minBundleSize, long startOffset, long endOffset,
-      String schema, Class<T> type, String codec, byte[] syncMarker, String fileSchema) {
-    super(fileName, minBundleSize, startOffset, endOffset);
-    this.readSchemaString = schema;
-    this.codec = codec;
-    this.syncMarker = syncMarker;
-    this.type = type;
-    this.fileSchemaString = fileSchema;
-  }
-
-  @Override
-  public void validate() {
-    // AvroSource objects do not need to be configured with more than a file pattern. Overridden to
-    // make this explicit.
-    super.validate();
-  }
-
-  @Override
-  public BlockBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) {
-    byte[] syncMarker = this.syncMarker;
-    String codec = this.codec;
-    String readSchemaString = this.readSchemaString;
-    String fileSchemaString = this.fileSchemaString;
-    // codec and syncMarker are initially null when the source is created, as they differ
-    // across input files and must be read from the file. Here, when we are creating a source
-    // for a subrange of a file, we can initialize these values. When the resulting AvroSource
-    // is further split, they do not need to be read again.
-    if (codec == null || syncMarker == null || fileSchemaString == null) {
-      AvroMetadata metadata;
-      try {
-        Collection<String> files = FileBasedSource.expandFilePattern(fileName);
-        Preconditions.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);
-      }
-      codec = metadata.getCodec();
-      syncMarker = metadata.getSyncMarker();
-      fileSchemaString = metadata.getSchemaString();
-      // If the source was created with a null schema, use the schema that we read from the file's
-      // metadata.
-      if (readSchemaString == null) {
-        readSchemaString = metadata.getSchemaString();
-      }
-    }
-    return new AvroSource<T>(fileName, getMinBundleSize(), start, end, readSchemaString, type,
-        codec, syncMarker, fileSchemaString);
-  }
-
-  @Override
-  protected BlockBasedReader<T> createSingleFileReader(PipelineOptions options) {
-    return new AvroReader<T>(this);
-  }
-
-  @Override
-  public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-    return false;
-  }
-
-  @Override
-  public AvroCoder<T> getDefaultOutputCoder() {
-    if (coder == null) {
-      Schema.Parser parser = new Schema.Parser();
-      coder = AvroCoder.of(type, parser.parse(readSchemaString));
-    }
-    return coder;
-  }
-
-  public String getSchema() {
-    return readSchemaString;
-  }
-
-  private Schema getReadSchema() {
-    if (readSchemaString == null) {
-      return null;
-    }
-
-    // If the schema has not been parsed, parse it.
-    if (readSchema == null) {
-      Schema.Parser parser = new Schema.Parser();
-      readSchema = parser.parse(readSchemaString);
-    }
-    return readSchema;
-  }
-
-  private Schema getFileSchema() {
-    if (fileSchemaString == null) {
-      return null;
-    }
-
-    // If the schema has not been parsed, parse it.
-    if (fileSchema == null) {
-      Schema.Parser parser = new Schema.Parser();
-      fileSchema = parser.parse(fileSchemaString);
-    }
-    return fileSchema;
-  }
-
-  private byte[] getSyncMarker() {
-    return syncMarker;
-  }
-
-  private String getCodec() {
-    return codec;
-  }
-
-  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);
-    if (type == GenericRecord.class) {
-      return new GenericDatumReader<>(fileSchema, readSchema);
-    } else {
-      return new ReflectDatumReader<>(fileSchema, readSchema);
-    }
-  }
-
-  /**
-   * A {@link BlockBasedSource.Block} of Avro records.
-   *
-   * @param <T> The type of records stored in the block.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  static class AvroBlock<T> extends Block<T> {
-    // The number of records in the block.
-    private final long numRecords;
-
-    // The current record in the block.
-    private T currentRecord;
-
-    // The index of the current record in the block.
-    private long currentRecordIndex = 0;
-
-    // A DatumReader to read records from the block.
-    private final DatumReader<T> reader;
-
-    // A BinaryDecoder used by the reader to decode records.
-    private final BinaryDecoder decoder;
-
-    /**
-     * Decodes a byte array as an InputStream. The byte array may be compressed using some
-     * codec. Reads from the returned stream will result in decompressed bytes.
-     *
-     * <p>This supports the same codecs as Avro's {@link CodecFactory}, namely those defined in
-     * {@link DataFileConstants}.
-     *
-     * <ul>
-     * <li>"snappy" : Google's Snappy compression
-     * <li>"deflate" : deflate compression
-     * <li>"bzip2" : Bzip2 compression
-     * <li>"xz" : xz compression
-     * <li>"null" (the string, not the value): Uncompressed data
-     * </ul>
-     */
-    private static InputStream decodeAsInputStream(byte[] data, String codec) throws IOException {
-      ByteArrayInputStream byteStream = new ByteArrayInputStream(data);
-      switch (codec) {
-        case DataFileConstants.SNAPPY_CODEC:
-          return new SnappyCompressorInputStream(byteStream);
-        case DataFileConstants.DEFLATE_CODEC:
-          // nowrap == true: Do not expect ZLIB header or checksum, as Avro does not write them.
-          Inflater inflater = new Inflater(true);
-          return new InflaterInputStream(byteStream, inflater);
-        case DataFileConstants.XZ_CODEC:
-          return new XZCompressorInputStream(byteStream);
-        case DataFileConstants.BZIP2_CODEC:
-          return new BZip2CompressorInputStream(byteStream);
-        case DataFileConstants.NULL_CODEC:
-          return byteStream;
-        default:
-          throw new IllegalArgumentException("Unsupported codec: " + codec);
-      }
-    }
-
-    AvroBlock(byte[] data, long numRecords, AvroSource<T> source) throws IOException {
-      this.numRecords = numRecords;
-      this.reader = source.createDatumReader();
-      this.decoder =
-          DecoderFactory.get().binaryDecoder(decodeAsInputStream(data, source.getCodec()), null);
-    }
-
-    @Override
-    public T getCurrentRecord() {
-      return currentRecord;
-    }
-
-    @Override
-    public boolean readNextRecord() throws IOException {
-      if (currentRecordIndex >= numRecords) {
-        return false;
-      }
-      currentRecord = reader.read(null, decoder);
-      currentRecordIndex++;
-      return true;
-    }
-
-    @Override
-    public double getFractionOfBlockConsumed() {
-      return ((double) currentRecordIndex) / numRecords;
-    }
-  }
-
-  /**
-   * A {@link BlockBasedSource.BlockBasedReader} for reading blocks from Avro files.
-   *
-   * <p>An Avro Object Container File consists of a header followed by a 16-bit sync marker
-   * and then a sequence of blocks, where each block begins with two encoded longs representing
-   * the total number of records in the block and the block's size in bytes, followed by the
-   * block's (optionally-encoded) records. Each block is terminated by a 16-bit sync marker.
-   *
-   * <p>Here, we consider the sync marker that precedes a block to be its offset, as this allows
-   * a reader that begins reading at that offset to detect the sync marker and the beginning of
-   * the block.
-   *
-   * @param <T> The type of records contained in the block.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  public static class AvroReader<T> extends BlockBasedReader<T> {
-    // The current block.
-    private AvroBlock<T> currentBlock;
-
-    // Offset of the block.
-    private long currentBlockOffset = 0;
-
-    // Size of the current block.
-    private long currentBlockSizeBytes = 0;
-
-    // Current offset within the stream.
-    private long currentOffset = 0;
-
-    // Stream used to read from the underlying file.
-    // A pushback stream is used to restore bytes buffered during seeking/decoding.
-    private PushbackInputStream stream;
-
-    // Small buffer for reading encoded values from the stream.
-    // The maximum size of an encoded long is 10 bytes, and this buffer will be used to read two.
-    private final byte[] readBuffer = new byte[20];
-
-    // Decoder to decode binary-encoded values from the buffer.
-    private BinaryDecoder decoder;
-
-    /**
-     * Reads Avro records of type {@code T} from the specified source.
-     */
-    public AvroReader(AvroSource<T> source) {
-      super(source);
-    }
-
-    @Override
-    public synchronized AvroSource<T> getCurrentSource() {
-      return (AvroSource<T>) super.getCurrentSource();
-    }
-
-    @Override
-    public boolean readNextBlock() throws IOException {
-      // The next block in the file is after the first sync marker that can be read starting from
-      // the current offset. First, we seek past the next sync marker, if it exists. After a sync
-      // marker is the start of a block. A block begins with the number of records contained in
-      // the block, encoded as a long, followed by the size of the block in bytes, encoded as a
-      // long. The currentOffset after this method should be last byte after this block, and the
-      // currentBlockOffset should be the start of the sync marker before this block.
-
-      // Seek to the next sync marker, if one exists.
-      currentOffset += advancePastNextSyncMarker(stream, getCurrentSource().getSyncMarker());
-
-      // The offset of the current block includes its preceding sync marker.
-      currentBlockOffset = currentOffset - getCurrentSource().getSyncMarker().length;
-
-      // Read a small buffer to parse the block header.
-      // We cannot use a BinaryDecoder to do this directly from the stream because a BinaryDecoder
-      // internally buffers data and we only want to read as many bytes from the stream as the size
-      // of the header. Though BinaryDecoder#InputStream returns an input stream that is aware of
-      // its internal buffering, we would have to re-wrap this input stream to seek for the next
-      // block in the file.
-      int read = stream.read(readBuffer);
-      // We reached the last sync marker in the file.
-      if (read <= 0) {
-        return false;
-      }
-      decoder = DecoderFactory.get().binaryDecoder(readBuffer, decoder);
-      long numRecords = decoder.readLong();
-      long blockSize = decoder.readLong();
-
-      // The decoder buffers data internally, but since we know the size of the stream the
-      // decoder has constructed from the readBuffer, the number of bytes available in the
-      // input stream is equal to the number of unconsumed bytes.
-      int headerSize = readBuffer.length - decoder.inputStream().available();
-      stream.unread(readBuffer, headerSize, read - headerSize);
-
-      // Create the current block by reading blockSize bytes. Block sizes permitted by the Avro
-      // specification are [32, 2^30], so this narrowing is ok.
-      byte[] data = new byte[(int) blockSize];
-      stream.read(data);
-      currentBlock = new AvroBlock<>(data, numRecords, getCurrentSource());
-      currentBlockSizeBytes = blockSize;
-
-      // Update current offset with the number of bytes we read to get the next block.
-      currentOffset += headerSize + blockSize;
-      return true;
-    }
-
-    @Override
-    public AvroBlock<T> getCurrentBlock() {
-      return currentBlock;
-    }
-
-    @Override
-    public long getCurrentBlockOffset() {
-      return currentBlockOffset;
-    }
-
-    @Override
-    public long getCurrentBlockSize() {
-      return currentBlockSizeBytes;
-    }
-
-    /**
-     * Creates a {@link PushbackInputStream} that has a large enough pushback buffer to be able
-     * to push back the syncBuffer and the readBuffer.
-     */
-    private PushbackInputStream createStream(ReadableByteChannel channel) {
-      return new PushbackInputStream(
-          Channels.newInputStream(channel),
-          getCurrentSource().getSyncMarker().length + readBuffer.length);
-    }
-
-    /**
-     * Starts reading from the provided channel. Assumes that the channel is already seeked to
-     * the source's start offset.
-     */
-    @Override
-    protected void startReading(ReadableByteChannel channel) throws IOException {
-      stream = createStream(channel);
-      currentOffset = getCurrentSource().getStartOffset();
-    }
-
-    /**
-     * Advances to the first byte after the next occurrence of the sync marker in the
-     * stream when reading from the current offset. Returns the number of bytes consumed
-     * from the stream. Note that this method requires a PushbackInputStream with a buffer
-     * at least as big as the marker it is seeking for.
-     */
-    static long advancePastNextSyncMarker(PushbackInputStream stream, byte[] syncMarker)
-        throws IOException {
-      Seeker seeker = new Seeker(syncMarker);
-      byte[] syncBuffer = new byte[syncMarker.length];
-      long totalBytesConsumed = 0;
-      // Seek until either a sync marker is found or we reach the end of the file.
-      int mark = -1; // Position of the last byte in the sync marker.
-      int read; // Number of bytes read.
-      do {
-        read = stream.read(syncBuffer);
-        if (read >= 0) {
-          mark = seeker.find(syncBuffer, read);
-          // Update the currentOffset with the number of bytes read.
-          totalBytesConsumed += read;
-        }
-      } while (mark < 0 && read > 0);
-
-      // If the sync marker was found, unread block data and update the current offsets.
-      if (mark >= 0) {
-        // The current offset after this call should be just past the sync marker, so we should
-        // unread the remaining buffer contents and update the currentOffset accordingly.
-        stream.unread(syncBuffer, mark + 1, read - (mark + 1));
-        totalBytesConsumed = totalBytesConsumed - (read - (mark + 1));
-      }
-      return totalBytesConsumed;
-    }
-
-    /**
-     * A {@link Seeker} looks for a given marker within a byte buffer. Uses naive string matching
-     * with a sliding window, as sync markers are small and random.
-     */
-    static class Seeker {
-      // The marker to search for.
-      private byte[] marker;
-
-      // Buffer used for the sliding window.
-      private byte[] searchBuffer;
-
-      // Number of bytes available to be matched in the buffer.
-      private int available = 0;
-
-      /**
-       * Create a {@link Seeker} that looks for the given marker.
-       */
-      public Seeker(byte[] marker) {
-        this.marker = marker;
-        this.searchBuffer = new byte[marker.length];
-      }
-
-      /**
-       * Find the marker in the byte buffer. Returns the index of the end of the marker in the
-       * buffer. If the marker is not found, returns -1.
-       *
-       * <p>State is maintained between calls. If the marker was partially matched, a subsequent
-       * call to find will resume matching the marker.
-       *
-       * @param buffer
-       * @return the index of the end of the marker within the buffer, or -1 if the buffer was not
-       * found.
-       */
-      public int find(byte[] buffer, int length) {
-        for (int i = 0; i < length; i++) {
-          System.arraycopy(searchBuffer, 1, searchBuffer, 0, searchBuffer.length - 1);
-          searchBuffer[searchBuffer.length - 1] = buffer[i];
-          available = Math.min(available + 1, searchBuffer.length);
-          if (ByteBuffer.wrap(searchBuffer, searchBuffer.length - available, available)
-                  .equals(ByteBuffer.wrap(marker))) {
-            available = 0;
-            return i;
-          }
-        }
-        return -1;
-      }
-    }
-  }
-}


[70/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 cedd696..594d2b8 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
@@ -15,30 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.common.DataflowExampleOptions;
+import org.apache.beam.examples.common.DataflowExampleUtils;
+import org.apache.beam.examples.complete.game.utils.WriteToBigQuery;
+import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.DateTimeZone;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index 7babc6e..6a6c1cf 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -15,27 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.complete.game.utils.WriteToBigQuery;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+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.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.apache.avro.reflect.Nullable;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
index 18ff0a3..f8c3e33 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game.injector;
+package org.apache.beam.examples.complete.game.injector;
 
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.model.PublishRequest;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 db58650..dbb5b8f 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game.injector;
+package org.apache.beam.examples.complete.game.injector;
 
 
 import com.google.api.client.googleapis.auth.oauth2.GoogleCredential;
@@ -28,7 +28,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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9d58837..3183a05 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game.injector;
+package org.apache.beam.examples.complete.game.injector;
 
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.http.HttpBackOffIOExceptionHandler;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
index 4bcfb72..f0b3afa 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
@@ -15,25 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game.utils;
+package org.apache.beam.examples.complete.game.utils;
+
+import org.apache.beam.examples.complete.game.UserScore;
+import org.apache.beam.sdk.Pipeline;
+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.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
 
 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.cloud.dataflow.examples.complete.game.UserScore;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
 
 import java.io.Serializable;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
index 41257ca..540c366 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game.utils;
+package org.apache.beam.examples.complete.game.utils;
+
+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.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
index b34b70d..ae92559 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.FlatMapElements;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
index 5832c89..df8800d 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
+package org.apache.beam.examples.complete.game;
 
-import com.google.cloud.dataflow.examples.complete.game.GameStats.CalculateSpammyUsers;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.examples.complete.game.GameStats.CalculateSpammyUsers;
+import org.apache.beam.sdk.Pipeline;
+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.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
index 3fd2c57..aa11c6c 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.complete.game.UserScore.GameActionInfo;
+import org.apache.beam.examples.complete.game.UserScore.ParseEventFn;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.joda.time.Instant;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
index b907ae7..40740a6 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
@@ -15,23 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ExtractAndSumScore;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.complete.game.UserScore.ExtractAndSumScore;
+import org.apache.beam.examples.complete.game.UserScore.GameActionInfo;
+import org.apache.beam.examples.complete.game.UserScore.ParseEventFn;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 59fb5e9..3eb1327 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
@@ -19,40 +19,40 @@ package org.apache.beam.runners.flink.examples;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+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.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TupleTag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7d12fed..04e11c1 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
@@ -19,15 +19,15 @@ package org.apache.beam.runners.flink.examples;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+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.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 public class WordCount {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 32dd046..2bc3490 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
@@ -19,20 +19,20 @@ package org.apache.beam.runners.flink.examples.streaming;
 
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.*;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+import org.apache.beam.sdk.io.*;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.Partition.PartitionFn;
+import org.apache.beam.sdk.transforms.windowing.*;
+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;
 import org.joda.time.Duration;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 894e3e0..8c8490b 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
@@ -19,21 +19,21 @@ package org.apache.beam.runners.flink.examples.streaming;
 
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+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;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.*;
+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;
 import org.joda.time.Duration;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 8fca1d4..95210ad 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
@@ -19,16 +19,16 @@ package org.apache.beam.runners.flink.examples.streaming;
 
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.Read;
+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;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.*;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7eb69ba..9f77f8c 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
@@ -19,15 +19,15 @@ package org.apache.beam.runners.flink.examples.streaming;
 
 import org.apache.beam.runners.flink.FlinkPipelineRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.*;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.*;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.*;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 6f93478..d31d790 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,11 +17,13 @@
  */
 package org.apache.beam.runners.flink;
 
-import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator;
 import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator;
+import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator;
 import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator;
-import com.google.cloud.dataflow.sdk.Pipeline;
+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;
@@ -115,7 +117,7 @@ public class FlinkPipelineExecutionEnvironment {
   /**
    * Depending on if the job is a Streaming or a Batch one, this method creates
    * the necessary execution environment and pipeline translator, and translates
-   * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
+   * the {@link org.apache.beam.sdk.values.PCollection} program into
    * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
    * one.
    * */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index 2f4b3ea..bfb0d6a 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -18,13 +18,14 @@
 package org.apache.beam.runners.flink;
 
 
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
+
 import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
index 4f53e35..bb016fa 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
@@ -17,17 +17,18 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
+
 import org.apache.flink.api.common.JobExecutionResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,7 +46,7 @@ import java.util.Map;
  * pipeline by first translating them to a Flink Plan and then executing them either locally
  * or on a Flink cluster, depending on the configuration.
  * <p>
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
+ * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
  */
 public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
index 3e30ab9..cd99f4e 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
@@ -18,13 +18,12 @@
 
 package org.apache.beam.runners.flink;
 
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
 import com.google.auto.service.AutoService;
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
 import com.google.common.collect.ImmutableList;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
index 8fd08ec..481d867 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
@@ -17,18 +17,18 @@
  */
 package org.apache.beam.runners.flink;
 
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
 
 import java.util.Collections;
 import java.util.Map;
 
 /**
- * Result of executing a {@link com.google.cloud.dataflow.sdk.Pipeline} with Flink. This
+ * Result of executing a {@link org.apache.beam.sdk.Pipeline} with Flink. This
  * has methods to query to job runtime and the final values of
- * {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s.
+ * {@link org.apache.beam.sdk.transforms.Aggregator}s.
  */
 public class FlinkRunnerResult implements PipelineResult {
   

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
index 71e3b54..9c36c21 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
@@ -17,12 +17,12 @@
  */
 package org.apache.beam.runners.flink.io;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
 
 /**
- * Transform for printing the contents of a {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * Transform for printing the contents of a {@link org.apache.beam.sdk.values.PCollection}.
  * to standard output.
  *
  * This is Flink-specific and will only work when executed using the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
index 28a10b7..5ce828c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -17,19 +17,20 @@
  */
 package org.apache.beam.runners.flink.translation;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.values.PValue;
+
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineTranslator}
  */
 public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
index b3c0cea..a7544da 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
@@ -31,37 +31,39 @@ import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
 import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat;
 import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+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.View;
+import org.apache.beam.sdk.transforms.Write;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGbkResultSchema;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+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.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.Write;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.Lists;
+
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.operators.Keys;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.io.AvroInputFormat;
 import org.apache.flink.api.java.io.AvroOutputFormat;
@@ -88,7 +90,7 @@ import java.util.Map;
 
 /**
  * Translators for transforming
- * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to
+ * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to
  * Flink {@link org.apache.flink.api.java.DataSet}s
  */
 public class FlinkBatchTransformTranslators {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
index 2294318..71950cf 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
@@ -19,16 +19,17 @@ package org.apache.beam.runners.flink.translation;
 
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TypedPValue;
+
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
index 9407bf5..82d23b0 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
@@ -17,14 +17,14 @@
  */
 package org.apache.beam.runners.flink.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline;
 
 /**
  * The role of this class is to translate the Beam operators to
  * their Flink counterparts. If we have a streaming job, this is instantiated as a
  * {@link FlinkStreamingPipelineTranslator}. In other case, i.e. for a batch job,
  * a {@link FlinkBatchPipelineTranslator} is created. Correspondingly, the
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based user-provided job is translated into
+ * {@link org.apache.beam.sdk.values.PCollection}-based user-provided job is translated into
  * a {@link org.apache.flink.streaming.api.datastream.DataStream} (for streaming) or a
  * {@link org.apache.flink.api.java.DataSet} (for batch) one.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index ac96807..4359842 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -17,22 +17,22 @@
  */
 package org.apache.beam.runners.flink.translation;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
+
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * This is a {@link FlinkPipelineTranslator} for streaming jobs. Its role is to translate the user-provided
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based job into a
+ * {@link org.apache.beam.sdk.values.PCollection}-based job into a
  * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
  *
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineTranslator}
  * */
 public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
index 2b9b0ee..541cd40 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
@@ -20,30 +20,48 @@ package org.apache.beam.runners.flink.translation;
 
 import org.apache.beam.runners.flink.translation.functions.UnionCoder;
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.*;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupByKeyWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundMultiWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundWrapper;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+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.join.RawUnionValue;
+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.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+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.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.Lists;
+
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.streaming.api.datastream.*;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.functions.IngestionTimeExtractor;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
@@ -52,7 +70,11 @@ import org.slf4j.LoggerFactory;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.util.*;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * This class contains all the mappings between Beam and Flink

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f6bdecd..8bc7317 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,13 +17,15 @@
  */
 package org.apache.beam.runners.flink.translation;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+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.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
index d5562b8..8e7cdd7 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
@@ -17,11 +17,12 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGbkResultSchema;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
 import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.util.Collector;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
index 56af397..e5ac748 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
@@ -18,7 +18,8 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
+
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.util.Collector;
 
@@ -26,7 +27,7 @@ import java.io.ByteArrayInputStream;
 import java.util.List;
 
 /**
- * This is a hack for transforming a {@link com.google.cloud.dataflow.sdk.transforms.Create}
+ * This is a hack for transforming a {@link org.apache.beam.sdk.transforms.Create}
  * operation. Flink does not allow {@code null} in it's equivalent operation:
  * {@link org.apache.flink.api.java.ExecutionEnvironment#fromElements(Object[])}. Therefore
  * we use a DataSource with one dummy element and output the elements of the Create operation

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index fe77e64..9ed5c7c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -18,22 +18,25 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+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.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+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.collect.ImmutableList;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.flink.api.common.functions.RichMapPartitionFunction;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
@@ -46,7 +49,7 @@ import java.util.Collection;
 import java.util.List;
 
 /**
- * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn}
+ * Encapsulates a {@link org.apache.beam.sdk.transforms.DoFn}
  * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
  */
 public class FlinkDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, OUT> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
index f92f888..7c7084d 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
@@ -17,7 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.values.KV;
+
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.util.Collector;
 
@@ -25,8 +26,8 @@ import java.util.Iterator;
 
 /**
  * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects
+ * {@link org.apache.beam.sdk.transforms.GroupByKey} operation. This reads the input
+ * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and collects
  * the values in a {@code List}.
  */
 public class FlinkKeyedListAggregationFunction<K,V> implements GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
index ca667ee..b1c4be6 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
@@ -18,20 +18,23 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+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.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.collect.ImmutableList;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.flink.api.common.functions.RichMapPartitionFunction;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
@@ -44,10 +47,10 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn} that uses side outputs
+ * Encapsulates a {@link org.apache.beam.sdk.transforms.DoFn} that uses side outputs
  * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
  *
- * We get a mapping from {@link com.google.cloud.dataflow.sdk.values.TupleTag} to output index
+ * We get a mapping from {@link org.apache.beam.sdk.values.TupleTag} to output index
  * and must tag all outputs with the output number. Afterwards a filter will filter out
  * those elements that are not to be in a specific output.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
index 37de37e..58a36b2 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
@@ -17,7 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.util.Collector;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
index 2de681b..a2bab2b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
@@ -17,8 +17,9 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.values.KV;
+
 import org.apache.flink.api.common.functions.GroupCombineFunction;
 import org.apache.flink.util.Collector;
 
@@ -26,8 +27,8 @@ import java.util.Iterator;
 
 /**
  * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated
+ * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input
+ * {@link org.apache.beam.sdk.values.KV} elements VI, extracts the key and emits accumulated
  * values which have the intermediate format VA.
  */
 public class FlinkPartialReduceFunction<K, VI, VA> implements GroupCombineFunction<KV<K, VI>, KV<K, VA>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
index 29193a2..43e458f 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
@@ -17,9 +17,11 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.values.KV;
+
 import com.google.common.collect.ImmutableList;
+
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.util.Collector;
 
@@ -27,8 +29,8 @@ import java.util.Iterator;
 
 /**
  * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and merges the
+ * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input
+ * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and merges the
  * accumulators resulting from the PartialReduce which produced the input VA.
  */
 public class FlinkReduceFunction<K, VA, VO> implements GroupReduceFunction<KV<K, VA>, KV<K, VO>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
index ac476d3..cc6fd8b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
@@ -18,14 +18,15 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -35,7 +36,7 @@ import java.util.List;
 /**
  * A UnionCoder encodes RawUnionValues.
  *
- * This file copied from {@link com.google.cloud.dataflow.sdk.transforms.join.UnionCoder}
+ * This file copied from {@link org.apache.beam.sdk.transforms.join.UnionCoder}
  */
 @SuppressWarnings("serial")
 public class UnionCoder extends StandardCoder<RawUnionValue> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
index 1249036..e06741c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
@@ -17,7 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.types;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
+
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -28,7 +29,7 @@ import java.io.ObjectInputStream;
 
 /**
  * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link com.google.cloud.dataflow.sdk.coders.Coder}.
+ * {@link org.apache.beam.sdk.coders.Coder}.
  */
 public class CoderComparator<T> extends TypeComparator<T> {
 



[07/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
deleted file mode 100644
index 0989fb5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
+++ /dev/null
@@ -1,44 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * A {@link ExecutorServiceFactory} that produces cached thread pools via
- * {@link Executors#newCachedThreadPool()}.
- */
-class CachedThreadPoolExecutorServiceFactory
-    implements DefaultValueFactory<ExecutorServiceFactory>, ExecutorServiceFactory {
-  private static final CachedThreadPoolExecutorServiceFactory INSTANCE =
-      new CachedThreadPoolExecutorServiceFactory();
-
-  @Override
-  public ExecutorServiceFactory create(PipelineOptions options) {
-    return INSTANCE;
-  }
-
-  @Override
-  public ExecutorService create() {
-    return Executors.newCachedThreadPool();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/Clock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/Clock.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/Clock.java
deleted file mode 100644
index 31c34ad..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/Clock.java
+++ /dev/null
@@ -1,31 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import org.joda.time.Instant;
-
-/**
- * Access to the current time.
- */
-public interface Clock {
-  /**
-   * Returns the current time as an {@link Instant}.
-   */
-  Instant now();
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CompletionCallback.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CompletionCallback.java
deleted file mode 100644
index b581616..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/CompletionCallback.java
+++ /dev/null
@@ -1,36 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-
-/**
- * A callback for completing a bundle of input.
- */
-interface CompletionCallback {
-  /**
-   * Handle a successful result, returning the committed outputs of the result.
-   */
-  Iterable<? extends CommittedBundle<?>> handleResult(
-      CommittedBundle<?> inputBundle, InProcessTransformResult result);
-
-  /**
-   * Handle a result that terminated abnormally due to the provided {@link Throwable}.
-   */
-  void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
deleted file mode 100644
index 48836e9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
+++ /dev/null
@@ -1,175 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
- * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
- * input after the upstream transform has produced and committed output.
- */
-public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
-  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
-  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
-  private Collection<PCollectionView<?>> views = new ArrayList<>();
-  private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
-  private Set<PValue> toFinalize = new HashSet<>();
-  private int numTransforms = 0;
-  private boolean finalized = false;
-
-  @Override
-  public void enterCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s "
-            + "which has already visited a Pipeline and is finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    if (node.isRootNode()) {
-      finalized = true;
-    }
-  }
-
-  @Override
-  public void visitTransform(TransformTreeNode node) {
-    toFinalize.removeAll(node.getInput().expand());
-    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
-    stepNames.put(appliedTransform, genStepName());
-    if (node.getInput().expand().isEmpty()) {
-      rootTransforms.add(appliedTransform);
-    } else {
-      for (PValue value : node.getInput().expand()) {
-        valueToConsumers.get(value).add(appliedTransform);
-      }
-    }
-  }
-
-  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
-        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
-    return application;
-  }
-
-  @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
-    toFinalize.add(value);
-    for (PValue expandedValue : value.expand()) {
-      valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
-      if (expandedValue instanceof PCollectionView) {
-        views.add((PCollectionView<?>) expandedValue);
-      }
-      expandedValue.recordAsOutput(getAppliedTransform(producer));
-    }
-    value.recordAsOutput(getAppliedTransform(producer));
-  }
-
-  private String genStepName() {
-    return String.format("s%s", numTransforms++);
-  }
-
-
-  /**
-   * Returns a mapping of each fully-expanded {@link PValue} to each
-   * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
-   * returned from {@code getValueToCustomers().get(PValue)},
-   * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
-   */
-  public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
-    checkState(
-        finalized,
-        "Can't call getValueToConsumers before the Pipeline has been completely traversed");
-
-    return valueToConsumers;
-  }
-
-  /**
-   * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
-   * name.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
-    checkState(
-        finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
-
-    return stepNames;
-  }
-
-  /**
-   * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
-   * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
-    checkState(
-        finalized,
-        "Can't call getRootTransforms before the Pipeline has been completely traversed");
-
-    return rootTransforms;
-  }
-
-  /**
-   * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
-   * {@link Pipeline}.
-   */
-  public Collection<PCollectionView<?>> getViews() {
-    checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
-
-    return views;
-  }
-
-  /**
-   * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
-   * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
-   * {@link Pipeline} is executed.
-   */
-  public Set<PValue> getUnfinalizedPValues() {
-    checkState(
-        finalized,
-        "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
-
-    return toFinalize;
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EmptyTransformEvaluator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EmptyTransformEvaluator.java
deleted file mode 100644
index 74aad19..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EmptyTransformEvaluator.java
+++ /dev/null
@@ -1,51 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-/**
- * A {@link TransformEvaluator} that ignores all input and produces no output. The result of
- * invoking {@link #finishBundle()} on this evaluator is to return an
- * {@link InProcessTransformResult} with no elements and a timestamp hold equal to
- * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}. Because the result contains no elements, this hold
- * will not affect the watermark.
- */
-final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
-  public static <T> TransformEvaluator<T> create(AppliedPTransform<?, ?, ?> transform) {
-    return new EmptyTransformEvaluator<T>(transform);
-  }
-
-  private final AppliedPTransform<?, ?, ?> transform;
-
-  private EmptyTransformEvaluator(AppliedPTransform<?, ?, ?> transform) {
-    this.transform = transform;
-  }
-
-  @Override
-  public void processElement(WindowedValue<T> element) throws Exception {}
-
-  @Override
-  public InProcessTransformResult finishBundle() throws Exception {
-    return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
-        .build();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
deleted file mode 100644
index 63a43b5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
+++ /dev/null
@@ -1,71 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-/**
- * Enforces that all elements in a {@link PCollection} can be encoded using that
- * {@link PCollection PCollection's} {@link Coder}.
- */
-class EncodabilityEnforcementFactory implements ModelEnforcementFactory {
-  public static EncodabilityEnforcementFactory create() {
-    return new EncodabilityEnforcementFactory();
-  }
-
-  @Override
-  public <T> ModelEnforcement<T> forBundle(
-      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
-    return new EncodabilityEnforcement<>(input);
-  }
-
-  private static class EncodabilityEnforcement<T> extends AbstractModelEnforcement<T> {
-    private Coder<T> coder;
-
-    public EncodabilityEnforcement(CommittedBundle<T> input) {
-      coder = SerializableUtils.clone(input.getPCollection().getCoder());
-    }
-
-    @Override
-    public void beforeElement(WindowedValue<T> element) {
-      try {
-        T clone = CoderUtils.clone(coder, element.getValue());
-        if (coder.consistentWithEquals()) {
-          checkArgument(
-              coder.structuralValue(element.getValue()).equals(coder.structuralValue(clone)),
-              "Coder %s of class %s does not maintain structural value equality"
-                  + " on input element %s",
-              coder,
-              coder.getClass().getSimpleName(),
-              element.getValue());
-        }
-      } catch (Exception e) {
-        throw UserCodeException.wrap(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EvaluatorKey.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EvaluatorKey.java
deleted file mode 100644
index 2e56918..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EvaluatorKey.java
+++ /dev/null
@@ -1,57 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-
-import java.util.Objects;
-
-/**
- * A (Transform, Pipeline Execution) key for stateful evaluators.
- *
- * Source evaluators are stateful to ensure data is not read multiple times. Evaluators are cached
- * to ensure that the reader is not restarted if the evaluator is retriggered. An
- * {@link EvaluatorKey} is used to ensure that multiple Pipelines can be executed without sharing
- * the same evaluators.
- */
-final class EvaluatorKey {
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final InProcessEvaluationContext context;
-
-  public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
-    this.transform = transform;
-    this.context = context;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(transform, context);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null || !(other instanceof EvaluatorKey)) {
-      return false;
-    }
-    EvaluatorKey that = (EvaluatorKey) other;
-    return Objects.equals(this.transform, that.transform)
-        && Objects.equals(this.context, that.context);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceFactory.java
deleted file mode 100644
index b0781aa..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceFactory.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import java.util.concurrent.ExecutorService;
-
-/**
- * A factory that creates {@link ExecutorService ExecutorServices}.
- * {@link ExecutorService ExecutorServices} created by this factory should be independent of one
- * another (e.g., if any executor is shut down the remaining executors should continue to process
- * work).
- */
-public interface ExecutorServiceFactory {
-  /**
-   * Create a new {@link ExecutorService}.
-   */
-  ExecutorService create();
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
deleted file mode 100644
index c770735..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
+++ /dev/null
@@ -1,478 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItem;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItems;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Optional;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
- * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
- */
-final class ExecutorServiceParallelExecutor implements InProcessExecutor {
-  private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
-
-  private final ExecutorService executorService;
-
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
-  private final Set<PValue> keyedPValues;
-  private final TransformEvaluatorRegistry registry;
-  @SuppressWarnings("rawtypes")
-  private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-      transformEnforcements;
-
-  private final InProcessEvaluationContext evaluationContext;
-
-  private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
-  private final ConcurrentMap<TransformExecutor<?>, Boolean> scheduledExecutors;
-
-  private final Queue<ExecutorUpdate> allUpdates;
-  private final BlockingQueue<VisibleExecutorUpdate> visibleUpdates;
-
-  private final TransformExecutorService parallelExecutorService;
-  private final CompletionCallback defaultCompletionCallback;
-
-  private Collection<AppliedPTransform<?, ?, ?>> rootNodes;
-
-  public static ExecutorServiceParallelExecutor create(
-      ExecutorService executorService,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Set<PValue> keyedPValues,
-      TransformEvaluatorRegistry registry,
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
-    return new ExecutorServiceParallelExecutor(
-        executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
-  }
-
-  private ExecutorServiceParallelExecutor(
-      ExecutorService executorService,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Set<PValue> keyedPValues,
-      TransformEvaluatorRegistry registry,
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
-    this.executorService = executorService;
-    this.valueToConsumers = valueToConsumers;
-    this.keyedPValues = keyedPValues;
-    this.registry = registry;
-    this.transformEnforcements = transformEnforcements;
-    this.evaluationContext = context;
-
-    scheduledExecutors = new ConcurrentHashMap<>();
-    // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
-    // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
-    // which stops the TransformExecutorServices from being prematurely garbage collected
-    executorServices =
-        CacheBuilder.newBuilder().weakValues().build(serialTransformExecutorServiceCacheLoader());
-
-    this.allUpdates = new ConcurrentLinkedQueue<>();
-    this.visibleUpdates = new ArrayBlockingQueue<>(20);
-
-    parallelExecutorService =
-        TransformExecutorServices.parallel(executorService, scheduledExecutors);
-    defaultCompletionCallback = new DefaultCompletionCallback();
-  }
-
-  private CacheLoader<StepAndKey, TransformExecutorService>
-      serialTransformExecutorServiceCacheLoader() {
-    return new CacheLoader<StepAndKey, TransformExecutorService>() {
-      @Override
-      public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
-        return TransformExecutorServices.serial(executorService, scheduledExecutors);
-      }
-    };
-  }
-
-  @Override
-  public void start(Collection<AppliedPTransform<?, ?, ?>> roots) {
-    rootNodes = ImmutableList.copyOf(roots);
-    Runnable monitorRunnable = new MonitorRunnable();
-    executorService.submit(monitorRunnable);
-  }
-
-  @SuppressWarnings("unchecked")
-  public void scheduleConsumption(
-      AppliedPTransform<?, ?, ?> consumer,
-      @Nullable CommittedBundle<?> bundle,
-      CompletionCallback onComplete) {
-    evaluateBundle(consumer, bundle, onComplete);
-  }
-
-  private <T> void evaluateBundle(
-      final AppliedPTransform<?, ?, ?> transform,
-      @Nullable final CommittedBundle<T> bundle,
-      final CompletionCallback onComplete) {
-    TransformExecutorService transformExecutor;
-
-    if (bundle != null && isKeyed(bundle.getPCollection())) {
-      final StepAndKey stepAndKey =
-          StepAndKey.of(transform, bundle == null ? null : bundle.getKey());
-      // This executor will remain reachable until it has executed all scheduled transforms.
-      // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
-      // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
-      // (scheduled due to the completion of another TransformExecutor) are provided to the
-      // ExecutorService before the Earlier TransformExecutor callable completes.
-      transformExecutor = executorServices.getUnchecked(stepAndKey);
-    } else {
-      transformExecutor = parallelExecutorService;
-    }
-
-    Collection<ModelEnforcementFactory> enforcements =
-        MoreObjects.firstNonNull(
-            transformEnforcements.get(transform.getTransform().getClass()),
-            Collections.<ModelEnforcementFactory>emptyList());
-
-    TransformExecutor<T> callable =
-        TransformExecutor.create(
-            registry,
-            enforcements,
-            evaluationContext,
-            bundle,
-            transform,
-            onComplete,
-            transformExecutor);
-    transformExecutor.schedule(callable);
-  }
-
-  private boolean isKeyed(PValue pvalue) {
-    return keyedPValues.contains(pvalue);
-  }
-
-  private void scheduleConsumers(CommittedBundle<?> bundle) {
-    for (AppliedPTransform<?, ?, ?> consumer : valueToConsumers.get(bundle.getPCollection())) {
-      scheduleConsumption(consumer, bundle, defaultCompletionCallback);
-    }
-  }
-
-  @Override
-  public void awaitCompletion() throws Throwable {
-    VisibleExecutorUpdate update;
-    do {
-      update = visibleUpdates.take();
-      if (update.throwable.isPresent()) {
-        throw update.throwable.get();
-      }
-    } while (!update.isDone());
-    executorService.shutdown();
-  }
-
-  /**
-   * The default {@link CompletionCallback}. The default completion callback is used to complete
-   * transform evaluations that are triggered due to the arrival of elements from an upstream
-   * transform, or for a source transform.
-   */
-  private class DefaultCompletionCallback implements CompletionCallback {
-    @Override
-    public Iterable<? extends CommittedBundle<?>> handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
-      Iterable<? extends CommittedBundle<?>> resultBundles =
-          evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
-      for (CommittedBundle<?> outputBundle : resultBundles) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
-      }
-      return resultBundles;
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
-      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
-    }
-  }
-
-  /**
-   * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
-   * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
-   * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
-   * as part of the result.
-   */
-  private class TimerCompletionCallback implements CompletionCallback {
-    private final Iterable<TimerData> timers;
-
-    private TimerCompletionCallback(Iterable<TimerData> timers) {
-      this.timers = timers;
-    }
-
-    @Override
-    public Iterable<? extends CommittedBundle<?>> handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
-      Iterable<? extends CommittedBundle<?>> resultBundles =
-          evaluationContext.handleResult(inputBundle, timers, result);
-      for (CommittedBundle<?> outputBundle : resultBundles) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
-      }
-      return resultBundles;
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
-      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
-    }
-  }
-
-  /**
-   * An internal status update on the state of the executor.
-   *
-   * Used to signal when the executor should be shut down (due to an exception).
-   */
-  private static class ExecutorUpdate {
-    private final Optional<? extends CommittedBundle<?>> bundle;
-    private final Optional<? extends Throwable> throwable;
-
-    public static ExecutorUpdate fromBundle(CommittedBundle<?> bundle) {
-      return new ExecutorUpdate(bundle, null);
-    }
-
-    public static ExecutorUpdate fromThrowable(Throwable t) {
-      return new ExecutorUpdate(null, t);
-    }
-
-    private ExecutorUpdate(CommittedBundle<?> producedBundle, Throwable throwable) {
-      this.bundle = Optional.fromNullable(producedBundle);
-      this.throwable = Optional.fromNullable(throwable);
-    }
-
-    public Optional<? extends CommittedBundle<?>> getBundle() {
-      return bundle;
-    }
-
-    public Optional<? extends Throwable> getException() {
-      return throwable;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(ExecutorUpdate.class)
-          .add("bundle", bundle)
-          .add("exception", throwable)
-          .toString();
-    }
-  }
-
-  /**
-   * An update of interest to the user. Used in {@link #awaitCompletion} to decide whether to
-   * return normally or throw an exception.
-   */
-  private static class VisibleExecutorUpdate {
-    private final Optional<? extends Throwable> throwable;
-    private final boolean done;
-
-    public static VisibleExecutorUpdate fromThrowable(Throwable e) {
-      return new VisibleExecutorUpdate(false, e);
-    }
-
-    public static VisibleExecutorUpdate finished() {
-      return new VisibleExecutorUpdate(true, null);
-    }
-
-    private VisibleExecutorUpdate(boolean done, @Nullable Throwable exception) {
-      this.throwable = Optional.fromNullable(exception);
-      this.done = done;
-    }
-
-    public boolean isDone() {
-      return done;
-    }
-  }
-
-  private class MonitorRunnable implements Runnable {
-    private final String runnableName =
-        String.format(
-            "%s$%s-monitor",
-            evaluationContext.getPipelineOptions().getAppName(),
-            ExecutorServiceParallelExecutor.class.getSimpleName());
-
-    @Override
-    public void run() {
-      String oldName = Thread.currentThread().getName();
-      Thread.currentThread().setName(runnableName);
-      try {
-        ExecutorUpdate update = allUpdates.poll();
-        // pull all of the pending work off of the queue
-        while (update != null) {
-          LOG.debug("Executor Update: {}", update);
-          if (update.getBundle().isPresent()) {
-            scheduleConsumers(update.getBundle().get());
-          } else if (update.getException().isPresent()) {
-            visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(update.getException().get()));
-          }
-          update = allUpdates.poll();
-        }
-        boolean timersFired = fireTimers();
-        addWorkIfNecessary(timersFired);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        LOG.error("Monitor died due to being interrupted");
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(e))) {
-          visibleUpdates.poll();
-        }
-      } catch (Throwable t) {
-        LOG.error("Monitor thread died due to throwable", t);
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(t))) {
-          visibleUpdates.poll();
-        }
-      } finally {
-        if (!shouldShutdown()) {
-          // The monitor thread should always be scheduled; but we only need to be scheduled once
-          executorService.submit(this);
-        }
-        Thread.currentThread().setName(oldName);
-      }
-    }
-
-    /**
-     * Fires any available timers. Returns true if at least one timer was fired.
-     */
-    private boolean fireTimers() throws Exception {
-      try {
-        boolean firedTimers = false;
-        for (Map.Entry<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> transformTimers :
-            evaluationContext.extractFiredTimers().entrySet()) {
-          AppliedPTransform<?, ?, ?> transform = transformTimers.getKey();
-          for (Map.Entry<Object, FiredTimers> keyTimers : transformTimers.getValue().entrySet()) {
-            for (TimeDomain domain : TimeDomain.values()) {
-              Collection<TimerData> delivery = keyTimers.getValue().getTimers(domain);
-              if (delivery.isEmpty()) {
-                continue;
-              }
-              KeyedWorkItem<Object, Object> work =
-                  KeyedWorkItems.timersWorkItem(keyTimers.getKey(), delivery);
-              @SuppressWarnings({"unchecked", "rawtypes"})
-              CommittedBundle<?> bundle =
-                  evaluationContext
-                      .createKeyedBundle(
-                          null, keyTimers.getKey(), (PCollection) transform.getInput())
-                      .add(WindowedValue.valueInEmptyWindows(work))
-                      .commit(Instant.now());
-              scheduleConsumption(transform, bundle, new TimerCompletionCallback(delivery));
-              firedTimers = true;
-            }
-          }
-        }
-        return firedTimers;
-      } catch (Exception e) {
-        LOG.error("Internal Error while delivering timers", e);
-        throw e;
-      }
-    }
-
-    private boolean shouldShutdown() {
-      if (evaluationContext.isDone()) {
-        LOG.debug("Pipeline is finished. Shutting down. {}");
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) {
-          visibleUpdates.poll();
-        }
-        executorService.shutdown();
-        return true;
-      }
-      return false;
-    }
-
-    /**
-     * If all active {@link TransformExecutor TransformExecutors} are in a blocked state,
-     * add more work from root nodes that may have additional work. This ensures that if a pipeline
-     * has elements available from the root nodes it will add those elements when necessary.
-     */
-    private void addWorkIfNecessary(boolean firedTimers) {
-      // If any timers have fired, they will add more work; We don't need to add more
-      if (firedTimers) {
-        return;
-      }
-      for (TransformExecutor<?> executor : scheduledExecutors.keySet()) {
-        if (!isExecutorBlocked(executor)) {
-          // We have at least one executor that can proceed without adding additional work
-          return;
-        }
-      }
-      // All current TransformExecutors are blocked; add more work from the roots.
-      for (AppliedPTransform<?, ?, ?> root : rootNodes) {
-        if (!evaluationContext.isDone(root)) {
-          scheduleConsumption(root, null, defaultCompletionCallback);
-        }
-      }
-    }
-
-    /**
-     * Return true if the provided executor might make more progress if no action is taken.
-     *
-     * <p>May return false even if all executor threads are currently blocked or cleaning up, as
-     * these can cause more work to be scheduled. If this does not occur, after these calls
-     * terminate, future calls will return true if all executors are waiting.
-     */
-    private boolean isExecutorBlocked(TransformExecutor<?> executor) {
-      Thread thread = executor.getThread();
-      if (thread == null) {
-        return false;
-      }
-      switch (thread.getState()) {
-        case TERMINATED:
-          throw new IllegalStateException(String.format(
-              "Unexpectedly encountered a Terminated TransformExecutor %s", executor));
-        case WAITING:
-        case TIMED_WAITING:
-          // The thread is waiting for some external input. Adding more work may cause the thread
-          // to stop waiting (e.g. the thread is waiting on an unbounded side input)
-          return true;
-        case BLOCKED:
-          // The executor is blocked on acquisition of a java monitor. This usually means it is
-          // making a call to the EvaluationContext, but not a model-blocking call - and will
-          // eventually complete, at which point we may reevaluate.
-        default:
-          // NEW and RUNNABLE threads can make progress
-          return false;
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/FlattenEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/FlattenEvaluatorFactory.java
deleted file mode 100644
index e7ad80c..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/FlattenEvaluatorFactory.java
+++ /dev/null
@@ -1,85 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Flatten.FlattenPCollectionList;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
- * {@link PTransform}.
- */
-class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
-            (AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
-      final AppliedPTransform<
-              PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
-          application,
-      final CommittedBundle<InputT> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
-    if (inputBundle == null) {
-      // it is impossible to call processElement on a flatten with no input bundle. A Flatten with
-      // no input bundle occurs as an output of Flatten.pcollections(PCollectionList.empty())
-      return new FlattenEvaluator<>(
-          null, StepTransformResult.withoutHold(application).build());
-    }
-    final UncommittedBundle<InputT> outputBundle =
-        evaluationContext.createBundle(inputBundle, application.getOutput());
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(application).addOutput(outputBundle).build();
-    return new FlattenEvaluator<>(outputBundle, result);
-  }
-
-  private static class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final UncommittedBundle<InputT> outputBundle;
-    private final InProcessTransformResult result;
-
-    public FlattenEvaluator(
-        UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
-      this.outputBundle = outputBundle;
-      this.result = result;
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> element) {
-      outputBundle.add(element);
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() {
-      return result;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java
deleted file mode 100644
index 9c46a41..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java
+++ /dev/null
@@ -1,56 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
-
-/**
- * A base class for implementing {@link PTransform} overrides, which behave identically to the
- * delegate transform but with overridden methods. Implementors are required to implement
- * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}.
- */
-public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
-    extends PTransform<InputT, OutputT> {
-  protected abstract PTransform<InputT, OutputT> delegate();
-
-  @Override
-  public OutputT apply(InputT input) {
-    return delegate().apply(input);
-  }
-
-  @Override
-  public void validate(InputT input) {
-    delegate().validate(input);
-  }
-
-  @Override
-  public String getName() {
-    return delegate().getName();
-  }
-
-  @Override
-  public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unused")
-      TypedPValue<T> output) throws CannotProvideCoderException {
-    return delegate().getDefaultOutputCoder(input, output);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
deleted file mode 100644
index 4b478ad..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
+++ /dev/null
@@ -1,273 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.StepTransformResult.Builder;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItem;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItemCoder;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItems;
-import com.google.cloud.dataflow.sdk.util.SystemReduceFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.annotations.VisibleForTesting;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKey}
- * {@link PTransform}.
- */
-class GroupByKeyEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = createEvaluator(
-            (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private <K, V> TransformEvaluator<KV<K, WindowedValue<V>>> createEvaluator(
-      final AppliedPTransform<
-              PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-              InProcessGroupByKeyOnly<K, V>>
-          application,
-      final CommittedBundle<KV<K, V>> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
-    return new GroupByKeyEvaluator<K, V>(evaluationContext, inputBundle, application);
-  }
-
-  private static class GroupByKeyEvaluator<K, V>
-      implements TransformEvaluator<KV<K, WindowedValue<V>>> {
-    private final InProcessEvaluationContext evaluationContext;
-
-    private final CommittedBundle<KV<K, V>> inputBundle;
-    private final AppliedPTransform<
-            PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-            InProcessGroupByKeyOnly<K, V>>
-        application;
-    private final Coder<K> keyCoder;
-    private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
-
-    public GroupByKeyEvaluator(
-        InProcessEvaluationContext evaluationContext,
-        CommittedBundle<KV<K, V>> inputBundle,
-        AppliedPTransform<
-                PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-                InProcessGroupByKeyOnly<K, V>>
-            application) {
-      this.evaluationContext = evaluationContext;
-      this.inputBundle = inputBundle;
-      this.application = application;
-
-      PCollection<KV<K, WindowedValue<V>>> input = application.getInput();
-      keyCoder = getKeyCoder(input.getCoder());
-      groupingMap = new HashMap<>();
-    }
-
-    private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
-      if (!(coder instanceof KvCoder)) {
-        throw new IllegalStateException();
-      }
-      @SuppressWarnings("unchecked")
-      Coder<K> keyCoder = ((KvCoder<K, WindowedValue<V>>) coder).getKeyCoder();
-      return keyCoder;
-    }
-
-    @Override
-    public void processElement(WindowedValue<KV<K, WindowedValue<V>>> element) {
-      KV<K, WindowedValue<V>> kv = element.getValue();
-      K key = kv.getKey();
-      byte[] encodedKey;
-      try {
-        encodedKey = encodeToByteArray(keyCoder, key);
-      } catch (CoderException exn) {
-        // TODO: Put in better element printing:
-        // truncate if too long.
-        throw new IllegalArgumentException(
-            String.format("unable to encode key %s of input to %s using %s", key, this, keyCoder),
-            exn);
-      }
-      GroupingKey<K> groupingKey = new GroupingKey<>(key, encodedKey);
-      List<WindowedValue<V>> values = groupingMap.get(groupingKey);
-      if (values == null) {
-        values = new ArrayList<WindowedValue<V>>();
-        groupingMap.put(groupingKey, values);
-      }
-      values.add(kv.getValue());
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() {
-      Builder resultBuilder = StepTransformResult.withoutHold(application);
-      for (Map.Entry<GroupingKey<K>, List<WindowedValue<V>>> groupedEntry :
-          groupingMap.entrySet()) {
-        K key = groupedEntry.getKey().key;
-        KeyedWorkItem<K, V> groupedKv =
-            KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
-        UncommittedBundle<KeyedWorkItem<K, V>> bundle =
-            evaluationContext.createKeyedBundle(inputBundle, key, application.getOutput());
-        bundle.add(WindowedValue.valueInEmptyWindows(groupedKv));
-        resultBuilder.addOutput(bundle);
-      }
-      return resultBuilder.build();
-    }
-
-    private static class GroupingKey<K> {
-      private K key;
-      private byte[] encodedKey;
-
-      public GroupingKey(K key, byte[] encodedKey) {
-        this.key = key;
-        this.encodedKey = encodedKey;
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (o instanceof GroupingKey) {
-          GroupingKey<?> that = (GroupingKey<?>) o;
-          return Arrays.equals(this.encodedKey, that.encodedKey);
-        } else {
-          return false;
-        }
-      }
-
-      @Override
-      public int hashCode() {
-        return Arrays.hashCode(encodedKey);
-      }
-    }
-  }
-
-  /**
-   * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
-   */
-  public static final class InProcessGroupByKeyOverrideFactory
-      implements PTransformOverrideFactory {
-    @Override
-    public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-        PTransform<InputT, OutputT> transform) {
-      if (transform instanceof GroupByKey) {
-        @SuppressWarnings({"rawtypes", "unchecked"})
-        PTransform<InputT, OutputT> override = new InProcessGroupByKey((GroupByKey) transform);
-        return override;
-      }
-      return transform;
-    }
-  }
-
-  /**
-   * An in-memory implementation of the {@link GroupByKey} primitive as a composite
-   * {@link PTransform}.
-   */
-  private static final class InProcessGroupByKey<K, V>
-      extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-    private final GroupByKey<K, V> original;
-
-    private InProcessGroupByKey(GroupByKey<K, V> from) {
-      this.original = from;
-    }
-
-    @Override
-    public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
-      return original;
-    }
-
-    @Override
-    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-      KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
-      // This operation groups by the combination of key and window,
-      // merging windows as needed, using the windows assigned to the
-      // key/value input elements and the window merge operation of the
-      // window function associated with the input PCollection.
-      WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
-      // Use the default GroupAlsoByWindow implementation
-      DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow =
-          groupAlsoByWindow(windowingStrategy, inputCoder.getValueCoder());
-
-      // By default, implement GroupByKey via a series of lower-level operations.
-      return input
-          // Make each input element's timestamp and assigned windows
-          // explicit, in the value part.
-          .apply(new ReifyTimestampsAndWindows<K, V>())
-
-          .apply(new InProcessGroupByKeyOnly<K, V>())
-          .setCoder(KeyedWorkItemCoder.of(inputCoder.getKeyCoder(),
-              inputCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
-
-          // Group each key's values by window, merging windows as needed.
-          .apply("GroupAlsoByWindow", ParDo.of(groupAlsoByWindow))
-
-          // And update the windowing strategy as appropriate.
-          .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
-          .setCoder(
-              KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
-    }
-
-    private <W extends BoundedWindow>
-        DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow(
-            final WindowingStrategy<?, W> windowingStrategy, final Coder<V> inputCoder) {
-      return GroupAlsoByWindowViaWindowSetDoFn.create(
-          windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
-    }
-  }
-
-  /**
-   * An implementation primitive to use in the evaluation of a {@link GroupByKey}
-   * {@link PTransform}.
-   */
-  public static final class InProcessGroupByKeyOnly<K, V>
-      extends PTransform<PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>> {
-    @Override
-    public PCollection<KeyedWorkItem<K, V>> apply(PCollection<KV<K, WindowedValue<V>>> input) {
-      return PCollection.<KeyedWorkItem<K, V>>createPrimitiveOutputInternal(
-          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
-    }
-
-    @VisibleForTesting
-    InProcessGroupByKeyOnly() {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
deleted file mode 100644
index 44670e8..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
+++ /dev/null
@@ -1,131 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.api.client.util.Throwables;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.MutationDetector;
-import com.google.cloud.dataflow.sdk.util.MutationDetectors;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.SetMultimap;
-
-import org.joda.time.Instant;
-
-/**
- * A {@link BundleFactory} that ensures that elements added to it are not mutated after being
- * output. Immutability checks are enforced at the time {@link UncommittedBundle#commit(Instant)} is
- * called, checking the value at that time against the value at the time the element was added. All
- * elements added to the bundle will be encoded by the {@link Coder} of the underlying
- * {@link PCollection}.
- *
- * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
- * after it is added to an output {@link PCollection}.
- */
-class ImmutabilityCheckingBundleFactory implements BundleFactory {
-  /**
-   * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
-   * {@link BundleFactory} to create the output bundle.
-   */
-  public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
-    return new ImmutabilityCheckingBundleFactory(underlying);
-  }
-
-  private final BundleFactory underlying;
-
-  private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
-    this.underlying = checkNotNull(underlying);
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createRootBundle(output));
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createBundle(input, output));
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(input, key, output));
-  }
-
-  private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
-    private final UncommittedBundle<T> underlying;
-    private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
-    private Coder<T> coder;
-
-    public ImmutabilityEnforcingBundle(UncommittedBundle<T> underlying) {
-      this.underlying = underlying;
-      mutationDetectors = HashMultimap.create();
-      coder = SerializableUtils.clone(getPCollection().getCoder());
-    }
-
-    @Override
-    public PCollection<T> getPCollection() {
-      return underlying.getPCollection();
-    }
-
-    @Override
-    public UncommittedBundle<T> add(WindowedValue<T> element) {
-      try {
-        mutationDetectors.put(
-            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
-      } catch (CoderException e) {
-        throw Throwables.propagate(e);
-      }
-      underlying.add(element);
-      return this;
-    }
-
-    @Override
-    public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
-      for (MutationDetector detector : mutationDetectors.values()) {
-        try {
-          detector.verifyUnmodified();
-        } catch (IllegalMutationException exn) {
-          throw UserCodeException.wrap(
-              new IllegalMutationException(
-                  String.format(
-                      "PTransform %s mutated value %s after it was output (new value was %s)."
-                          + " Values must not be mutated in any way after being output.",
-                      underlying.getPCollection().getProducingTransformInternal().getFullName(),
-                      exn.getSavedValue(),
-                      exn.getNewValue()),
-                  exn.getSavedValue(),
-                  exn.getNewValue(),
-                  exn));
-        }
-      }
-      return underlying.commit(synchronizedProcessingTime);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
deleted file mode 100644
index 8b7ccba..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
+++ /dev/null
@@ -1,104 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.MutationDetector;
-import com.google.cloud.dataflow.sdk.util.MutationDetectors;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-import java.util.IdentityHashMap;
-import java.util.Map;
-
-/**
- * {@link ModelEnforcement} that enforces elements are not modified over the course of processing
- * an element.
- *
- * <p>Implies {@link EncodabilityEnforcment}.
- */
-class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
-  public static ModelEnforcementFactory create() {
-    return new ImmutabilityEnforcementFactory();
-  }
-
-  @Override
-  public <T> ModelEnforcement<T> forBundle(
-      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
-    return new ImmutabilityCheckingEnforcement<T>(input, consumer);
-  }
-
-  private static class ImmutabilityCheckingEnforcement<T> extends AbstractModelEnforcement<T> {
-    private final AppliedPTransform<?, ?, ?> transform;
-    private final Map<WindowedValue<T>, MutationDetector> mutationElements;
-    private final Coder<T> coder;
-
-    private ImmutabilityCheckingEnforcement(
-        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> transform) {
-      this.transform = transform;
-      coder = SerializableUtils.clone(input.getPCollection().getCoder());
-      mutationElements = new IdentityHashMap<>();
-    }
-
-    @Override
-    public void beforeElement(WindowedValue<T> element) {
-      try {
-        mutationElements.put(
-            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
-      } catch (CoderException e) {
-        throw UserCodeException.wrap(e);
-      }
-    }
-
-    @Override
-    public void afterElement(WindowedValue<T> element) {
-      verifyUnmodified(mutationElements.get(element));
-    }
-
-    @Override
-    public void afterFinish(
-        CommittedBundle<T> input,
-        InProcessTransformResult result,
-        Iterable<? extends CommittedBundle<?>> outputs) {
-      for (MutationDetector detector : mutationElements.values()) {
-        verifyUnmodified(detector);
-      }
-    }
-
-    private void verifyUnmodified(MutationDetector detector) {
-      try {
-        detector.verifyUnmodified();
-      } catch (IllegalMutationException e) {
-        throw new IllegalMutationException(
-            String.format(
-                "PTransform %s illegaly mutated value %s of class %s."
-                    + " Input values must not be mutated in any way.",
-                transform.getFullName(),
-                e.getSavedValue(),
-                e.getSavedValue().getClass()),
-            e.getSavedValue(),
-            e.getNewValue());
-      }
-    }
-  }
-}


[04/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
deleted file mode 100644
index 1d6563d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
+++ /dev/null
@@ -1,92 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link BoundMulti} primitive {@link PTransform}.
- */
-class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator = (TransformEvaluator<T>) createMultiEvaluator(
-            (AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private static <InT, OuT> ParDoInProcessEvaluator<InT> createMultiEvaluator(
-      AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
-      CommittedBundle<InT> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    PCollectionTuple output = application.getOutput();
-    Map<TupleTag<?>, PCollection<?>> outputs = output.getAll();
-    Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
-    for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
-      outputBundles.put(
-          outputEntry.getKey(),
-          evaluationContext.createBundle(inputBundle, outputEntry.getValue()));
-    }
-    InProcessExecutionContext executionContext =
-        evaluationContext.getExecutionContext(application, inputBundle.getKey());
-    String stepName = evaluationContext.getStepName(application);
-    InProcessStepContext stepContext =
-        executionContext.getOrCreateStepContext(stepName, stepName, null);
-
-    CounterSet counters = evaluationContext.createCounterSet();
-
-    DoFn<InT, OuT> fn = application.getTransform().getFn();
-    DoFnRunner<InT, OuT> runner =
-        DoFnRunners.createDefault(
-            evaluationContext.getPipelineOptions(),
-            fn,
-            evaluationContext.createSideInputReader(application.getTransform().getSideInputs()),
-            BundleOutputManager.create(outputBundles),
-            application.getTransform().getMainOutputTag(),
-            application.getTransform().getSideOutputTags().getAll(),
-            stepContext,
-            counters.getAddCounterMutator(),
-            application.getInput().getWindowingStrategy());
-
-    runner.startBundle();
-
-    return new ParDoInProcessEvaluator<>(
-        runner, application, counters, outputBundles.values(), stepContext);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
deleted file mode 100644
index 28c86a9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
+++ /dev/null
@@ -1,89 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ParDoInProcessEvaluator.BundleOutputManager;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.Bound;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-import java.util.Collections;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound ParDo.Bound} primitive {@link PTransform}.
- */
-class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      final AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator = (TransformEvaluator<T>) createSingleEvaluator(
-            (AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private static <InputT, OutputT> ParDoInProcessEvaluator<InputT> createSingleEvaluator(
-      @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
-          Bound<InputT, OutputT>> application,
-      CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
-    TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
-    UncommittedBundle<OutputT> outputBundle =
-        evaluationContext.createBundle(inputBundle, application.getOutput());
-
-    InProcessExecutionContext executionContext =
-        evaluationContext.getExecutionContext(application, inputBundle.getKey());
-    String stepName = evaluationContext.getStepName(application);
-    InProcessStepContext stepContext =
-        executionContext.getOrCreateStepContext(stepName, stepName, null);
-
-    CounterSet counters = evaluationContext.createCounterSet();
-
-    DoFnRunner<InputT, OutputT> runner =
-        DoFnRunners.createDefault(
-            evaluationContext.getPipelineOptions(),
-            application.getTransform().getFn(),
-            evaluationContext.createSideInputReader(application.getTransform().getSideInputs()),
-            BundleOutputManager.create(
-                Collections.<TupleTag<?>, UncommittedBundle<?>>singletonMap(
-                    mainOutputTag, outputBundle)),
-            mainOutputTag,
-            Collections.<TupleTag<?>>emptyList(),
-            stepContext,
-            counters.getAddCounterMutator(),
-            application.getInput().getWindowingStrategy());
-
-    runner.startBundle();
-    return new ParDoInProcessEvaluator<InputT>(
-        runner,
-        application,
-        counters,
-        Collections.<UncommittedBundle<?>>singleton(outputBundle),
-        stepContext);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PassthroughTransformEvaluator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PassthroughTransformEvaluator.java
deleted file mode 100644
index 50d0ed9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PassthroughTransformEvaluator.java
+++ /dev/null
@@ -1,49 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
-  public static <InputT> PassthroughTransformEvaluator<InputT> create(
-      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
-    return new PassthroughTransformEvaluator<>(transform, output);
-  }
-
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final UncommittedBundle<InputT> output;
-
-  private PassthroughTransformEvaluator(
-      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
-    this.transform = transform;
-    this.output = output;
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> element) throws Exception {
-    output.add(element);
-  }
-
-  @Override
-  public InProcessTransformResult finishBundle() throws Exception {
-    return StepTransformResult.withoutHold(transform).addOutput(output).build();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ShardControlledWrite.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ShardControlledWrite.java
deleted file mode 100644
index fc6419e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepAndKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepAndKey.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepAndKey.java
deleted file mode 100644
index a6683fe..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepAndKey.java
+++ /dev/null
@@ -1,70 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.common.base.MoreObjects;
-
-import java.util.Objects;
-
-/**
- * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
- * per-step in a keyed manner (e.g. State).
- */
-final class StepAndKey {
-  private final AppliedPTransform<?, ?, ?> step;
-  private final Object key;
-
-  /**
-   * Create a new {@link StepAndKey} with the provided step and key.
-   */
-  public static StepAndKey of(AppliedPTransform<?, ?, ?> step, Object key) {
-    return new StepAndKey(step, key);
-  }
-
-  private StepAndKey(AppliedPTransform<?, ?, ?> step, Object key) {
-    this.step = step;
-    this.key = key;
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(StepAndKey.class)
-        .add("step", step.getFullName())
-        .add("key", key)
-        .toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(step, key);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == this) {
-      return true;
-    } else if (!(other instanceof StepAndKey)) {
-      return false;
-    } else {
-      StepAndKey that = (StepAndKey) other;
-      return Objects.equals(this.step, that.step)
-          && Objects.equals(this.key, that.key);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepTransformResult.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepTransformResult.java
deleted file mode 100644
index a1fd657..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/StepTransformResult.java
+++ /dev/null
@@ -1,164 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * An immutable {@link InProcessTransformResult}.
- */
-public class StepTransformResult implements InProcessTransformResult {
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final Iterable<? extends UncommittedBundle<?>> bundles;
-  @Nullable private final CopyOnAccessInMemoryStateInternals<?> state;
-  private final TimerUpdate timerUpdate;
-  @Nullable private final CounterSet counters;
-  private final Instant watermarkHold;
-
-  private StepTransformResult(
-      AppliedPTransform<?, ?, ?> transform,
-      Iterable<? extends UncommittedBundle<?>> outputBundles,
-      CopyOnAccessInMemoryStateInternals<?> state,
-      TimerUpdate timerUpdate,
-      CounterSet counters,
-      Instant watermarkHold) {
-    this.transform = checkNotNull(transform);
-    this.bundles = checkNotNull(outputBundles);
-    this.state = state;
-    this.timerUpdate = checkNotNull(timerUpdate);
-    this.counters = counters;
-    this.watermarkHold = checkNotNull(watermarkHold);
-  }
-
-  @Override
-  public Iterable<? extends UncommittedBundle<?>> getOutputBundles() {
-    return bundles;
-  }
-
-  @Override
-  public CounterSet getCounters() {
-    return counters;
-  }
-
-  @Override
-  public AppliedPTransform<?, ?, ?> getTransform() {
-    return transform;
-  }
-
-  @Override
-  public Instant getWatermarkHold() {
-    return watermarkHold;
-  }
-
-  @Nullable
-  @Override
-  public CopyOnAccessInMemoryStateInternals<?> getState() {
-    return state;
-  }
-
-  @Override
-  public TimerUpdate getTimerUpdate() {
-    return timerUpdate;
-  }
-
-  public static Builder withHold(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
-    return new Builder(transform, watermarkHold);
-  }
-
-  public static Builder withoutHold(AppliedPTransform<?, ?, ?> transform) {
-    return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(StepTransformResult.class)
-        .add("transform", transform)
-        .toString();
-  }
-
-  /**
-   * A builder for creating instances of {@link StepTransformResult}.
-   */
-  public static class Builder {
-    private final AppliedPTransform<?, ?, ?> transform;
-    private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
-    private CopyOnAccessInMemoryStateInternals<?> state;
-    private TimerUpdate timerUpdate;
-    private CounterSet counters;
-    private final Instant watermarkHold;
-
-    private Builder(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
-      this.transform = transform;
-      this.watermarkHold = watermarkHold;
-      this.bundlesBuilder = ImmutableList.builder();
-      this.timerUpdate = TimerUpdate.builder(null).build();
-    }
-
-    public StepTransformResult build() {
-      return new StepTransformResult(
-          transform,
-          bundlesBuilder.build(),
-          state,
-          timerUpdate,
-          counters,
-          watermarkHold);
-    }
-
-    public Builder withCounters(CounterSet counters) {
-      this.counters = counters;
-      return this;
-    }
-
-    public Builder withState(CopyOnAccessInMemoryStateInternals<?> state) {
-      this.state = state;
-      return this;
-    }
-
-    public Builder withTimerUpdate(TimerUpdate timerUpdate) {
-      this.timerUpdate = timerUpdate;
-      return this;
-    }
-
-    public Builder addOutput(
-        UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
-      bundlesBuilder.add(outputBundle);
-      bundlesBuilder.add(outputBundles);
-      return this;
-    }
-
-    public Builder addOutput(Collection<UncommittedBundle<?>> outputBundles) {
-      bundlesBuilder.addAll(outputBundles);
-      return this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TextIOShardedWriteFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TextIOShardedWriteFactory.java
deleted file mode 100644
index af433c4..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.TextIO.Write.Bound;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluator.java
deleted file mode 100644
index 007fd18..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluator.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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-/**
- * An evaluator of a specific application of a transform. Will be used for at least one
- * {@link CommittedBundle}.
- *
- * @param <InputT> the type of elements that will be passed to {@link #processElement}
- */
-public interface TransformEvaluator<InputT> {
-  /**
-   * Process an element in the input {@link CommittedBundle}.
-   *
-   * @param element the element to process
-   */
-  void processElement(WindowedValue<InputT> element) throws Exception;
-
-  /**
-   * Finish processing the bundle of this {@link TransformEvaluator}.
-   *
-   * After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
-   * and no more elements will be processed.
-   *
-   * @return an {@link InProcessTransformResult} containing the results of this bundle evaluation.
-   */
-  InProcessTransformResult finishBundle() throws Exception;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorFactory.java
deleted file mode 100644
index c5d657a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorFactory.java
+++ /dev/null
@@ -1,44 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory for creating instances of {@link TransformEvaluator} for the application of a
- * {@link PTransform}.
- */
-public interface TransformEvaluatorFactory {
-  /**
-   * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
-   *
-   * Any work that must be done before input elements are processed (such as calling
-   * {@link DoFn#startBundle(DoFn.Context)}) must be done before the {@link TransformEvaluator} is
-   * made available to the caller.
-   *
-   * @throws Exception whenever constructing the underlying evaluator throws an exception
-   */
-  <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorRegistry.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorRegistry.java
deleted file mode 100644
index 113bad8..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformEvaluatorRegistry.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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Flatten.FlattenPCollectionList;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
- * implementations based on the type of {@link PTransform} of the application.
- */
-class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
-  public static TransformEvaluatorRegistry defaultRegistry() {
-    @SuppressWarnings("rawtypes")
-    ImmutableMap<Class<? extends PTransform>, TransformEvaluatorFactory> primitives =
-        ImmutableMap.<Class<? extends PTransform>, TransformEvaluatorFactory>builder()
-            .put(Read.Bounded.class, new BoundedReadEvaluatorFactory())
-            .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory())
-            .put(ParDo.Bound.class, new ParDoSingleEvaluatorFactory())
-            .put(ParDo.BoundMulti.class, new ParDoMultiEvaluatorFactory())
-            .put(
-                GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly.class,
-                new GroupByKeyEvaluatorFactory())
-            .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory())
-            .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
-            .put(Window.Bound.class, new WindowEvaluatorFactory())
-            .build();
-    return new TransformEvaluatorRegistry(primitives);
-  }
-
-  // the TransformEvaluatorFactories can construct instances of all generic types of transform,
-  // so all instances of a primitive can be handled with the same evaluator factory.
-  @SuppressWarnings("rawtypes")
-  private final Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories;
-
-  private TransformEvaluatorRegistry(
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories) {
-    this.factories = factories;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws Exception {
-    TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
-    return factory.forApplication(application, inputBundle, evaluationContext);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutor.java
deleted file mode 100644
index 872aa80..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutor.java
+++ /dev/null
@@ -1,175 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.common.base.Throwables;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a
- * {@link TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering
- * the result using a registered {@link CompletionCallback}.
- *
- * <p>A {@link TransformExecutor} that is currently executing also provides access to the thread
- * that it is being executed on.
- */
-class TransformExecutor<T> implements Callable<InProcessTransformResult> {
-  public static <T> TransformExecutor<T> create(
-      TransformEvaluatorFactory factory,
-      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
-      CommittedBundle<T> inputBundle,
-      AppliedPTransform<?, ?, ?> transform,
-      CompletionCallback completionCallback,
-      TransformExecutorService transformEvaluationState) {
-    return new TransformExecutor<>(
-        factory,
-        modelEnforcements,
-        evaluationContext,
-        inputBundle,
-        transform,
-        completionCallback,
-        transformEvaluationState);
-  }
-
-  private final TransformEvaluatorFactory evaluatorFactory;
-  private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
-
-  private final InProcessEvaluationContext evaluationContext;
-
-  /** The transform that will be evaluated. */
-  private final AppliedPTransform<?, ?, ?> transform;
-  /** The inputs this {@link TransformExecutor} will deliver to the transform. */
-  private final CommittedBundle<T> inputBundle;
-
-  private final CompletionCallback onComplete;
-  private final TransformExecutorService transformEvaluationState;
-
-  private final AtomicReference<Thread> thread;
-
-  private TransformExecutor(
-      TransformEvaluatorFactory factory,
-      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
-      CommittedBundle<T> inputBundle,
-      AppliedPTransform<?, ?, ?> transform,
-      CompletionCallback completionCallback,
-      TransformExecutorService transformEvaluationState) {
-    this.evaluatorFactory = factory;
-    this.modelEnforcements = modelEnforcements;
-    this.evaluationContext = evaluationContext;
-
-    this.inputBundle = inputBundle;
-    this.transform = transform;
-
-    this.onComplete = completionCallback;
-
-    this.transformEvaluationState = transformEvaluationState;
-    this.thread = new AtomicReference<>();
-  }
-
-  @Override
-  public InProcessTransformResult call() {
-    checkState(
-        thread.compareAndSet(null, Thread.currentThread()),
-        "Tried to execute %s for %s on thread %s, but is already executing on thread %s",
-        TransformExecutor.class.getSimpleName(),
-        transform.getFullName(),
-        Thread.currentThread(),
-        thread.get());
-    try {
-      Collection<ModelEnforcement<T>> enforcements = new ArrayList<>();
-      for (ModelEnforcementFactory enforcementFactory : modelEnforcements) {
-        ModelEnforcement<T> enforcement = enforcementFactory.forBundle(inputBundle, transform);
-        enforcements.add(enforcement);
-      }
-      TransformEvaluator<T> evaluator =
-          evaluatorFactory.forApplication(transform, inputBundle, evaluationContext);
-
-      processElements(evaluator, enforcements);
-
-      InProcessTransformResult result = finishBundle(evaluator, enforcements);
-      return result;
-    } catch (Throwable t) {
-      onComplete.handleThrowable(inputBundle, t);
-      throw Throwables.propagate(t);
-    } finally {
-      transformEvaluationState.complete(this);
-    }
-  }
-
-  /**
-   * Processes all the elements in the input bundle using the transform evaluator, applying any
-   * necessary {@link ModelEnforcement ModelEnforcements}.
-   */
-  private void processElements(
-      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
-      throws Exception {
-    if (inputBundle != null) {
-      for (WindowedValue<T> value : inputBundle.getElements()) {
-        for (ModelEnforcement<T> enforcement : enforcements) {
-          enforcement.beforeElement(value);
-        }
-
-        evaluator.processElement(value);
-
-        for (ModelEnforcement<T> enforcement : enforcements) {
-          enforcement.afterElement(value);
-        }
-      }
-    }
-  }
-
-  /**
-   * Finishes processing the input bundle and commit the result using the
-   * {@link CompletionCallback}, applying any {@link ModelEnforcement} if necessary.
-   *
-   * @return the {@link InProcessTransformResult} produced by
-   *         {@link TransformEvaluator#finishBundle()}
-   */
-  private InProcessTransformResult finishBundle(
-      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
-      throws Exception {
-    InProcessTransformResult result = evaluator.finishBundle();
-    Iterable<? extends CommittedBundle<?>> outputs = onComplete.handleResult(inputBundle, result);
-    for (ModelEnforcement<T> enforcement : enforcements) {
-      enforcement.afterFinish(inputBundle, result, outputs);
-    }
-    return result;
-  }
-
-  /**
-   * If this {@link TransformExecutor} is currently executing, return the thread it is executing in.
-   * Otherwise, return null.
-   */
-  @Nullable
-  public Thread getThread() {
-    return thread.get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorService.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorService.java
deleted file mode 100644
index 795cd3f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorService.java
+++ /dev/null
@@ -1,36 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-/**
- * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
- * appropriate for the {@link StepAndKey} the executor exists for.
- */
-interface TransformExecutorService {
-  /**
-   * Schedule the provided work to be eventually executed.
-   */
-  void schedule(TransformExecutor<?> work);
-
-  /**
-   * Finish executing the provided work. This may cause additional
-   * {@link TransformExecutor TransformExecutors} to be evaluated.
-   */
-  void complete(TransformExecutor<?> completed);
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorServices.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorServices.java
deleted file mode 100644
index 194d080..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/TransformExecutorServices.java
+++ /dev/null
@@ -1,155 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.common.base.MoreObjects;
-
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Static factory methods for constructing instances of {@link TransformExecutorService}.
- */
-final class TransformExecutorServices {
-  private TransformExecutorServices() {
-    // Do not instantiate
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * parallel.
-   */
-  public static TransformExecutorService parallel(
-      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-    return new ParallelEvaluationState(executor, scheduled);
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * serial.
-   */
-  public static TransformExecutorService serial(
-      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-    return new SerialEvaluationState(executor, scheduled);
-  }
-
-  /**
-   * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
-   * scheduled will be immediately submitted to the {@link ExecutorService}.
-   *
-   * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
-   * processed in parallel.
-   */
-  private static class ParallelEvaluationState implements TransformExecutorService {
-    private final ExecutorService executor;
-    private final Map<TransformExecutor<?>, Boolean> scheduled;
-
-    private ParallelEvaluationState(
-        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-      this.executor = executor;
-      this.scheduled = scheduled;
-    }
-
-    @Override
-    public void schedule(TransformExecutor<?> work) {
-      executor.submit(work);
-      scheduled.put(work, true);
-    }
-
-    @Override
-    public void complete(TransformExecutor<?> completed) {
-      scheduled.remove(completed);
-    }
-  }
-
-  /**
-   * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
-   * scheduled will be placed on the work queue. Only one item of work will be submitted to the
-   * {@link ExecutorService} at any time.
-   *
-   * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair.
-   * Keyed computations are processed serially per step.
-   */
-  private static class SerialEvaluationState implements TransformExecutorService {
-    private final ExecutorService executor;
-    private final Map<TransformExecutor<?>, Boolean> scheduled;
-
-    private AtomicReference<TransformExecutor<?>> currentlyEvaluating;
-    private final Queue<TransformExecutor<?>> workQueue;
-
-    private SerialEvaluationState(
-        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-      this.scheduled = scheduled;
-      this.executor = executor;
-      this.currentlyEvaluating = new AtomicReference<>();
-      this.workQueue = new ConcurrentLinkedQueue<>();
-    }
-
-    /**
-     * Schedules the work, adding it to the work queue if there is a bundle currently being
-     * evaluated and scheduling it immediately otherwise.
-     */
-    @Override
-    public void schedule(TransformExecutor<?> work) {
-      workQueue.offer(work);
-      updateCurrentlyEvaluating();
-    }
-
-    @Override
-    public void complete(TransformExecutor<?> completed) {
-      if (!currentlyEvaluating.compareAndSet(completed, null)) {
-        throw new IllegalStateException(
-            "Finished work "
-                + completed
-                + " but could not complete due to unexpected currently executing "
-                + currentlyEvaluating.get());
-      }
-      scheduled.remove(completed);
-      updateCurrentlyEvaluating();
-    }
-
-    private void updateCurrentlyEvaluating() {
-      if (currentlyEvaluating.get() == null) {
-        // Only synchronize if we need to update what's currently evaluating
-        synchronized (this) {
-          TransformExecutor<?> newWork = workQueue.poll();
-          if (newWork != null) {
-            if (currentlyEvaluating.compareAndSet(null, newWork)) {
-              scheduled.put(newWork, true);
-              executor.submit(newWork);
-            } else {
-              workQueue.offer(newWork);
-            }
-          }
-        }
-      }
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(SerialEvaluationState.class)
-          .add("currentlyEvaluating", currentlyEvaluating)
-          .add("workQueue", workQueue)
-          .toString();
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
deleted file mode 100644
index 0f2e4f4..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,178 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.io.Read.Unbounded;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}.
- */
-class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
-  /*
-   * An evaluator for a Source is stateful, to ensure the CheckpointMark is properly persisted.
-   * Evaluators are cached here to ensure that the checkpoint mark is appropriately reused
-   * and any splits are honored.
-   */
-  private final ConcurrentMap<EvaluatorKey, Queue<? extends UnboundedReadEvaluator<?>>>
-      sourceEvaluators = new ConcurrentHashMap<>();
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
-    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
-  }
-
-  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
-      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    UnboundedReadEvaluator<?> currentEvaluator =
-        getTransformEvaluatorQueue(transform, evaluationContext).poll();
-    if (currentEvaluator == null) {
-      return EmptyTransformEvaluator.create(transform);
-    }
-    return currentEvaluator;
-  }
-
-  /**
-   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
-   * provided application of {@link Unbounded Read.Unbounded}, initializing it if required.
-   *
-   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
-   * already done so.
-   */
-  @SuppressWarnings("unchecked")
-  private <OutputT> Queue<UnboundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
-      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    // Key by the application and the context the evaluation is occurring in (which call to
-    // Pipeline#run).
-    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
-    @SuppressWarnings("unchecked")
-    Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue =
-        (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-    if (evaluatorQueue == null) {
-      evaluatorQueue = new ConcurrentLinkedQueue<>();
-      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
-        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
-        // factory for this transform
-        UnboundedSource<OutputT, ?> source = transform.getTransform().getSource();
-        UnboundedReadEvaluator<OutputT> evaluator =
-            new UnboundedReadEvaluator<OutputT>(
-                transform, evaluationContext, source, evaluatorQueue);
-        evaluatorQueue.offer(evaluator);
-      } else {
-        // otherwise return the existing Queue that arrived before us
-        evaluatorQueue = (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-      }
-    }
-    return evaluatorQueue;
-  }
-
-  /**
-   * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource},
-   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
-   * creates the {@link UnboundedReader} and consumes some currently available input.
-   *
-   * <p>Calls to {@link UnboundedReadEvaluator} are not internally thread-safe, and should only be
-   * used by a single thread at a time. Each {@link UnboundedReadEvaluator} maintains its own
-   * checkpoint, and constructs its reader from the current checkpoint in each call to
-   * {@link #finishBundle()}.
-   */
-  private static class UnboundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
-    private static final int ARBITRARY_MAX_ELEMENTS = 10;
-    private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
-    private final Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue;
-    /**
-     * The source being read from by this {@link UnboundedReadEvaluator}. This may not be the same
-     * source as derived from {@link #transform} due to splitting.
-     */
-    private final UnboundedSource<OutputT, ?> source;
-    private CheckpointMark checkpointMark;
-
-    public UnboundedReadEvaluator(
-        AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
-        UnboundedSource<OutputT, ?> source,
-        Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue) {
-      this.transform = transform;
-      this.evaluationContext = evaluationContext;
-      this.evaluatorQueue = evaluatorQueue;
-      this.source = source;
-      this.checkpointMark = null;
-    }
-
-    @Override
-    public void processElement(WindowedValue<Object> element) {}
-
-    @Override
-    public InProcessTransformResult finishBundle() throws IOException {
-      UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
-      try (UnboundedReader<OutputT> reader =
-              createReader(source, evaluationContext.getPipelineOptions());) {
-        int numElements = 0;
-        if (reader.start()) {
-          do {
-            output.add(
-                WindowedValue.timestampedValueInGlobalWindow(
-                    reader.getCurrent(), reader.getCurrentTimestamp()));
-            numElements++;
-          } while (numElements < ARBITRARY_MAX_ELEMENTS && reader.advance());
-        }
-        checkpointMark = reader.getCheckpointMark();
-        checkpointMark.finalizeCheckpoint();
-        // TODO: When exercising create initial splits, make this the minimum watermark across all
-        // existing readers
-        StepTransformResult result =
-            StepTransformResult.withHold(transform, reader.getWatermark())
-                .addOutput(output)
-                .build();
-        evaluatorQueue.offer(this);
-        return result;
-      }
-    }
-
-    private <CheckpointMarkT extends CheckpointMark> UnboundedReader<OutputT> createReader(
-        UnboundedSource<OutputT, CheckpointMarkT> source, PipelineOptions options) {
-      @SuppressWarnings("unchecked")
-      CheckpointMarkT mark = (CheckpointMarkT) checkpointMark;
-      return source.createReader(options, mark);
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ViewEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ViewEvaluatorFactory.java
deleted file mode 100644
index 112ea0c..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ViewEvaluatorFactory.java
+++ /dev/null
@@ -1,145 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link CreatePCollectionView} primitive {@link PTransform}.
- *
- * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
- * the {@link WriteView} {@link PTransform}, which is part of the
- * {@link InProcessCreatePCollectionView} composite transform. This transform is an override for the
- * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
- * written.
- */
-class ViewEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      InProcessPipelineRunner.CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator = createEvaluator(
-            (AppliedPTransform) application, evaluationContext);
-    return evaluator;
-  }
-
-  private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
-      final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
-          application,
-      InProcessEvaluationContext context) {
-    PCollection<Iterable<InT>> input = application.getInput();
-    final PCollectionViewWriter<InT, OuT> writer =
-        context.createPCollectionViewWriter(input, application.getOutput());
-    return new TransformEvaluator<Iterable<InT>>() {
-      private final List<WindowedValue<InT>> elements = new ArrayList<>();
-
-      @Override
-      public void processElement(WindowedValue<Iterable<InT>> element) {
-        for (InT input : element.getValue()) {
-          elements.add(element.withValue(input));
-        }
-      }
-
-      @Override
-      public InProcessTransformResult finishBundle() {
-        writer.add(elements);
-        return StepTransformResult.withoutHold(application).build();
-      }
-    };
-  }
-
-  public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
-    @Override
-    public <InputT extends PInput, OutputT extends POutput>
-        PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
-      if (transform instanceof CreatePCollectionView) {
-
-      }
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      PTransform<InputT, OutputT> createView =
-          (PTransform<InputT, OutputT>)
-              new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
-      return createView;
-    }
-  }
-
-  /**
-   * An in-process override for {@link CreatePCollectionView}.
-   */
-  private static class InProcessCreatePCollectionView<ElemT, ViewT>
-      extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
-    private final CreatePCollectionView<ElemT, ViewT> og;
-
-    private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
-      this.og = og;
-    }
-
-    @Override
-    public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
-      return input.apply(WithKeys.<Void, ElemT>of((Void) null))
-          .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()))
-          .apply(GroupByKey.<Void, ElemT>create())
-          .apply(Values.<Iterable<ElemT>>create())
-          .apply(new WriteView<ElemT, ViewT>(og));
-    }
-
-    @Override
-    protected PTransform<PCollection<ElemT>, PCollectionView<ViewT>> delegate() {
-      return og;
-    }
-  }
-
-  /**
-   * An in-process implementation of the {@link CreatePCollectionView} primitive.
-   *
-   * This implementation requires the input {@link PCollection} to be an iterable, which is provided
-   * to {@link PCollectionView#fromIterableInternal(Iterable)}.
-   */
-  public static final class WriteView<ElemT, ViewT>
-      extends PTransform<PCollection<Iterable<ElemT>>, PCollectionView<ViewT>> {
-    private final CreatePCollectionView<ElemT, ViewT> og;
-
-    WriteView(CreatePCollectionView<ElemT, ViewT> og) {
-      this.og = og;
-    }
-
-    @Override
-    public PCollectionView<ViewT> apply(PCollection<Iterable<ElemT>> input) {
-      return og.getView();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WatermarkCallbackExecutor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WatermarkCallbackExecutor.java
deleted file mode 100644
index 230985d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WatermarkCallbackExecutor.java
+++ /dev/null
@@ -1,145 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.Ordering;
-
-import org.joda.time.Instant;
-
-import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Executes callbacks that occur based on the progression of the watermark per-step.
- *
- * <p>Callbacks are registered by calls to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)},
- * and are executed after a call to {@link #fireForWatermark(AppliedPTransform, Instant)} with the
- * same {@link AppliedPTransform} and a watermark sufficient to ensure that the trigger for the
- * windowing strategy would have been produced.
- *
- * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
- * {@link AppliedPTransform} - any call to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)}
- * that could have potentially already fired should be followed by a call to
- * {@link #fireForWatermark(AppliedPTransform, Instant)} for the same transform with the current
- * value of the watermark.
- */
-class WatermarkCallbackExecutor {
-  /**
-   * Create a new {@link WatermarkCallbackExecutor}.
-   */
-  public static WatermarkCallbackExecutor create() {
-    return new WatermarkCallbackExecutor();
-  }
-
-  private final ConcurrentMap<AppliedPTransform<?, ?, ?>, PriorityQueue<WatermarkCallback>>
-      callbacks;
-  private final ExecutorService executor;
-
-  private WatermarkCallbackExecutor() {
-    this.callbacks = new ConcurrentHashMap<>();
-    this.executor = Executors.newSingleThreadExecutor();
-  }
-
-  /**
-   * Execute the provided {@link Runnable} after the next call to
-   * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window is guaranteed to have
-   * produced output.
-   */
-  public void callOnGuaranteedFiring(
-      AppliedPTransform<?, ?, ?> step,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    WatermarkCallback callback =
-        WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
-
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
-      if (callbacks.putIfAbsent(step, callbackQueue) != null) {
-        callbackQueue = callbacks.get(step);
-      }
-    }
-
-    synchronized (callbackQueue) {
-      callbackQueue.offer(callback);
-    }
-  }
-
-  /**
-   * Schedule all pending callbacks that must have produced output by the time of the provided
-   * watermark.
-   */
-  public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark) {
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      return;
-    }
-    synchronized (callbackQueue) {
-      while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
-        executor.submit(callbackQueue.poll().getCallback());
-      }
-    }
-  }
-
-  private static class WatermarkCallback {
-    public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
-        BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
-      @SuppressWarnings("unchecked")
-      Instant firingAfter =
-          strategy.getTrigger().getSpec().getWatermarkThatGuaranteesFiring((W) window);
-      return new WatermarkCallback(firingAfter, callback);
-    }
-
-    private final Instant fireAfter;
-    private final Runnable callback;
-
-    private WatermarkCallback(Instant fireAfter, Runnable callback) {
-      this.fireAfter = fireAfter;
-      this.callback = callback;
-    }
-
-    public boolean shouldFire(Instant currentWatermark) {
-      return currentWatermark.isAfter(fireAfter)
-          || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    }
-
-    public Runnable getCallback() {
-      return callback;
-    }
-  }
-
-  private static class CallbackOrdering extends Ordering<WatermarkCallback> {
-    @Override
-    public int compare(WatermarkCallback left, WatermarkCallback right) {
-      return ComparisonChain.start()
-          .compare(left.fireAfter, right.fireAfter)
-          .compare(left.callback, right.callback, Ordering.arbitrary())
-          .result();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WindowEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WindowEvaluatorFactory.java
deleted file mode 100644
index e553dbb..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/WindowEvaluatorFactory.java
+++ /dev/null
@@ -1,131 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound Window.Bound} primitive {@link PTransform}.
- */
-class WindowEvaluatorFactory implements TransformEvaluatorFactory {
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws Exception {
-    return createTransformEvaluator(
-        (AppliedPTransform) application, inputBundle, evaluationContext);
-  }
-
-  private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
-      AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
-    UncommittedBundle<InputT> outputBundle =
-        evaluationContext.createBundle(inputBundle, transform.getOutput());
-    if (fn == null) {
-      return PassthroughTransformEvaluator.create(transform, outputBundle);
-    }
-    return new WindowIntoEvaluator<>(transform, fn, outputBundle);
-  }
-
-  private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>>
-        transform;
-    private final WindowFn<InputT, ?> windowFn;
-    private final UncommittedBundle<InputT> outputBundle;
-
-    @SuppressWarnings("unchecked")
-    public WindowIntoEvaluator(
-        AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
-        WindowFn<? super InputT, ?> windowFn,
-        UncommittedBundle<InputT> outputBundle) {
-      this.outputBundle = outputBundle;
-      this.transform = transform;
-      // Safe contravariant cast
-      this.windowFn = (WindowFn<InputT, ?>) windowFn;
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> element) throws Exception {
-      Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
-      outputBundle.add(
-          WindowedValue.<InputT>of(
-              element.getValue(), element.getTimestamp(), windows, PaneInfo.NO_FIRING));
-    }
-
-    private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
-        WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
-      WindowFn<InputT, W>.AssignContext assignContext =
-          new InProcessAssignContext<>(windowFn, element);
-      Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
-      return windows;
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() throws Exception {
-      return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
-    }
-  }
-
-  private static class InProcessAssignContext<InputT, W extends BoundedWindow>
-      extends WindowFn<InputT, W>.AssignContext {
-    private final WindowedValue<InputT> value;
-
-    public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
-      fn.super();
-      this.value = value;
-    }
-
-    @Override
-    public InputT element() {
-      return value.getValue();
-    }
-
-    @Override
-    public Instant timestamp() {
-      return value.getTimestamp();
-    }
-
-    @Override
-    public Collection<? extends BoundedWindow> windows() {
-      return value.getWindows();
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java
deleted file mode 100644
index 5c24672..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java
+++ /dev/null
@@ -1,34 +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.
- */
-/**
- * Defines runners for executing Pipelines in different modes, including
- * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and
- * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
- *
- * <p>{@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline}
- * locally, without contacting the Dataflow service.
- * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} submits a
- * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine
- * instances. {@code DataflowPipelineRunner} returns
- * as soon as the {@code Pipeline} has been submitted. Use
- * {@link com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner} to have execution
- * updates printed to the console.
- *
- * <p>The runner is specified as part {@link com.google.cloud.dataflow.sdk.options.PipelineOptions}.
- */
-package com.google.cloud.dataflow.sdk.runners;


[46/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java
deleted file mode 100644
index 269ee6a..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java
+++ /dev/null
@@ -1,267 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Mean;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Logger;
-
-/**
- * This is an example that demonstrates several approaches to filtering, and use of the Mean
- * transform. It shows how to dynamically set parameters by defining and using new pipeline options,
- * and how to use a value derived by the pipeline.
- *
- * <p>Concepts: The Mean transform; Options configuration; using pipeline-derived data as a side
- * input; approaches to filtering, selection, and projection.
- *
- * <p>The example reads public samples of weather data from BigQuery. It performs a
- * projection on the data, finds the global mean of the temperature readings, filters on readings
- * for a single given month, and then outputs only data (for that month) that has a mean temp
- * smaller than the derived global mean.
-*
- * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
- * table.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- *   [--monthFilter=<month_number>]
- * }
- * </pre>
- * where optional parameter {@code --monthFilter} is set to a number 1-12.
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- *   [--monthFilter=<month_number>]
- * }
- * </pre>
- * where optional parameter {@code --monthFilter} is set to a number 1-12.
- *
- * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
- * and can be overridden with {@code --input}.
- */
-public class FilterExamples {
-  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
-  private static final String WEATHER_SAMPLES_TABLE =
-      "clouddataflow-readonly:samples.weather_stations";
-  static final Logger LOG = Logger.getLogger(FilterExamples.class.getName());
-  static final int MONTH_TO_FILTER = 7;
-
-  /**
-   * Examines each row in the input table. Outputs only the subset of the cells this example
-   * is interested in-- the mean_temp and year, month, and day-- as a bigquery table row.
-   */
-  static class ProjectionFn extends DoFn<TableRow, TableRow> {
-    @Override
-    public void processElement(ProcessContext c){
-      TableRow row = c.element();
-      // Grab year, month, day, mean_temp from the row
-      Integer year = Integer.parseInt((String) row.get("year"));
-      Integer month = Integer.parseInt((String) row.get("month"));
-      Integer day = Integer.parseInt((String) row.get("day"));
-      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
-      // Prepares the data for writing to BigQuery by building a TableRow object
-      TableRow outRow = new TableRow()
-          .set("year", year).set("month", month)
-          .set("day", day).set("mean_temp", meanTemp);
-      c.output(outRow);
-    }
-  }
-
-  /**
-   * Implements 'filter' functionality.
-   *
-   * <p>Examines each row in the input table. Outputs only rows from the month
-   * monthFilter, which is passed in as a parameter during construction of this DoFn.
-   */
-  static class FilterSingleMonthDataFn extends DoFn<TableRow, TableRow> {
-    Integer monthFilter;
-
-    public FilterSingleMonthDataFn(Integer monthFilter) {
-      this.monthFilter = monthFilter;
-    }
-
-    @Override
-    public void processElement(ProcessContext c){
-      TableRow row = c.element();
-      Integer month;
-      month = (Integer) row.get("month");
-      if (month.equals(this.monthFilter)) {
-        c.output(row);
-      }
-    }
-  }
-
-  /**
-   * Examines each row (weather reading) in the input table. Output the temperature
-   * reading for that row ('mean_temp').
-   */
-  static class ExtractTempFn extends DoFn<TableRow, Double> {
-    @Override
-    public void processElement(ProcessContext c){
-      TableRow row = c.element();
-      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
-      c.output(meanTemp);
-    }
-  }
-
-
-
-  /*
-   * Finds the global mean of the mean_temp for each day/record, and outputs
-   * only data that has a mean temp larger than this global mean.
-   **/
-  static class BelowGlobalMean
-      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
-    Integer monthFilter;
-
-    public BelowGlobalMean(Integer monthFilter) {
-      this.monthFilter = monthFilter;
-    }
-
-
-    @Override
-    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
-
-      // Extract the mean_temp from each row.
-      PCollection<Double> meanTemps = rows.apply(
-          ParDo.of(new ExtractTempFn()));
-
-      // Find the global mean, of all the mean_temp readings in the weather data,
-      // and prepare this singleton PCollectionView for use as a side input.
-      final PCollectionView<Double> globalMeanTemp =
-          meanTemps.apply(Mean.<Double>globally())
-               .apply(View.<Double>asSingleton());
-
-      // Rows filtered to remove all but a single month
-      PCollection<TableRow> monthFilteredRows = rows
-          .apply(ParDo.of(new FilterSingleMonthDataFn(monthFilter)));
-
-      // Then, use the global mean as a side input, to further filter the weather data.
-      // By using a side input to pass in the filtering criteria, we can use a value
-      // that is computed earlier in pipeline execution.
-      // We'll only output readings with temperatures below this mean.
-      PCollection<TableRow> filteredRows = monthFilteredRows
-          .apply(ParDo
-              .named("ParseAndFilter")
-              .withSideInputs(globalMeanTemp)
-              .of(new DoFn<TableRow, TableRow>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  Double meanTemp = Double.parseDouble(c.element().get("mean_temp").toString());
-                  Double gTemp = c.sideInput(globalMeanTemp);
-                  if (meanTemp < gTemp) {
-                    c.output(c.element());
-                  }
-                }
-              }));
-
-      return filteredRows;
-    }
-  }
-
-
-  /**
-   * Options supported by {@link FilterExamples}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Table to read from, specified as "
-        + "<project_id>:<dataset_id>.<table_id>")
-    @Default.String(WEATHER_SAMPLES_TABLE)
-    String getInput();
-    void setInput(String value);
-
-    @Description("Table to write to, specified as "
-        + "<project_id>:<dataset_id>.<table_id>. "
-        + "The dataset_id must already exist")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-
-    @Description("Numeric value of month to filter on")
-    @Default.Integer(MONTH_TO_FILTER)
-    Integer getMonthFilter();
-    void setMonthFilter(Integer value);
-  }
-
-  /**
-   * Helper method to build the table schema for the output table.
-   */
-  private static TableSchema buildWeatherSchemaProjection() {
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("year").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("day").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("mean_temp").setType("FLOAT"));
-    TableSchema schema = new TableSchema().setFields(fields);
-    return schema;
-  }
-
-  public static void main(String[] args)
-      throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-
-    TableSchema schema = buildWeatherSchemaProjection();
-
-    p.apply(BigQueryIO.Read.from(options.getInput()))
-     .apply(ParDo.of(new ProjectionFn()))
-     .apply(new BelowGlobalMean(options.getMonthFilter()))
-     .apply(BigQueryIO.Write
-        .to(options.getOutput())
-        .withSchema(schema)
-        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java
deleted file mode 100644
index 765420e..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java
+++ /dev/null
@@ -1,186 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-/**
- * This example shows how to do a join on two collections.
- * It uses a sample of the GDELT 'world event' data (http://goo.gl/OB6oin), joining the event
- * 'action' country code against a table that maps country codes to country names.
- *
- * <p>Concepts: Join operation; multiple input sources.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and a local output file or output prefix on GCS:
- * <pre>{@code
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
- */
-public class JoinExamples {
-
-  // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
-  private static final String GDELT_EVENTS_TABLE =
-      "clouddataflow-readonly:samples.gdelt_sample";
-  // A table that maps country codes to country names.
-  private static final String COUNTRY_CODES =
-      "gdelt-bq:full.crosswalk_geocountrycodetohuman";
-
-  /**
-   * Join two collections, using country code as the key.
-   */
-  static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
-      PCollection<TableRow> countryCodes) throws Exception {
-
-    final TupleTag<String> eventInfoTag = new TupleTag<String>();
-    final TupleTag<String> countryInfoTag = new TupleTag<String>();
-
-    // transform both input collections to tuple collections, where the keys are country
-    // codes in both cases.
-    PCollection<KV<String, String>> eventInfo = eventsTable.apply(
-        ParDo.of(new ExtractEventDataFn()));
-    PCollection<KV<String, String>> countryInfo = countryCodes.apply(
-        ParDo.of(new ExtractCountryInfoFn()));
-
-    // country code 'key' -> CGBKR (<event info>, <country name>)
-    PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
-        .of(eventInfoTag, eventInfo)
-        .and(countryInfoTag, countryInfo)
-        .apply(CoGroupByKey.<String>create());
-
-    // Process the CoGbkResult elements generated by the CoGroupByKey transform.
-    // country code 'key' -> string of <event info>, <country name>
-    PCollection<KV<String, String>> finalResultCollection =
-      kvpCollection.apply(ParDo.named("Process").of(
-        new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
-          @Override
-          public void processElement(ProcessContext c) {
-            KV<String, CoGbkResult> e = c.element();
-            String countryCode = e.getKey();
-            String countryName = "none";
-            countryName = e.getValue().getOnly(countryInfoTag);
-            for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
-              // Generate a string that combines information from both collection values
-              c.output(KV.of(countryCode, "Country name: " + countryName
-                      + ", Event info: " + eventInfo));
-            }
-          }
-      }));
-
-    // write to GCS
-    PCollection<String> formattedResults = finalResultCollection
-        .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
-          @Override
-          public void processElement(ProcessContext c) {
-            String outputstring = "Country code: " + c.element().getKey()
-                + ", " + c.element().getValue();
-            c.output(outputstring);
-          }
-        }));
-    return formattedResults;
-  }
-
-  /**
-   * Examines each row (event) in the input table. Output a KV with the key the country
-   * code of the event, and the value a string encoding event information.
-   */
-  static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = c.element();
-      String countryCode = (String) row.get("ActionGeo_CountryCode");
-      String sqlDate = (String) row.get("SQLDATE");
-      String actor1Name = (String) row.get("Actor1Name");
-      String sourceUrl = (String) row.get("SOURCEURL");
-      String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
-      c.output(KV.of(countryCode, eventInfo));
-    }
-  }
-
-
-  /**
-   * Examines each row (country info) in the input table. Output a KV with the key the country
-   * code, and the value the country name.
-   */
-  static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = c.element();
-      String countryCode = (String) row.get("FIPSCC");
-      String countryName = (String) row.get("HumanName");
-      c.output(KV.of(countryCode, countryName));
-    }
-  }
-
-
-  /**
-   * Options supported by {@link JoinExamples}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Path of the file to write to")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args) throws Exception {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-    // the following two 'applys' create multiple inputs to our pipeline, one for each
-    // of our two input sources.
-    PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
-    PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
-    PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
-    formattedResults.apply(TextIO.Write.to(options.getOutput()));
-    p.run();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java
deleted file mode 100644
index b35a57f..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java
+++ /dev/null
@@ -1,174 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * An example that reads the public samples of weather data from BigQuery, and finds
- * the maximum temperature ('mean_temp') for each month.
- *
- * <p>Concepts: The 'Max' statistical combination function, and how to find the max per
- * key group.
- *
- * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
- * table.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output, with the form
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations }
- * and can be overridden with {@code --input}.
- */
-public class MaxPerKeyExamples {
-  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
-  private static final String WEATHER_SAMPLES_TABLE =
-      "clouddataflow-readonly:samples.weather_stations";
-
-  /**
-   * Examines each row (weather reading) in the input table. Output the month of the reading,
-   * and the mean_temp.
-   */
-  static class ExtractTempFn extends DoFn<TableRow, KV<Integer, Double>> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = c.element();
-      Integer month = Integer.parseInt((String) row.get("month"));
-      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
-      c.output(KV.of(month, meanTemp));
-    }
-  }
-
-  /**
-   * Format the results to a TableRow, to save to BigQuery.
-   *
-   */
-  static class FormatMaxesFn extends DoFn<KV<Integer, Double>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = new TableRow()
-          .set("month", c.element().getKey())
-          .set("max_mean_temp", c.element().getValue());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Reads rows from a weather data table, and finds the max mean_temp for each
-   * month via the 'Max' statistical combination function.
-   */
-  static class MaxMeanTemp
-      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
-    @Override
-    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
-
-      // row... => <month, mean_temp> ...
-      PCollection<KV<Integer, Double>> temps = rows.apply(
-          ParDo.of(new ExtractTempFn()));
-
-      // month, mean_temp... => <month, max mean temp>...
-      PCollection<KV<Integer, Double>> tempMaxes =
-          temps.apply(Max.<Integer>doublesPerKey());
-
-      // <month, max>... => row...
-      PCollection<TableRow> results = tempMaxes.apply(
-          ParDo.of(new FormatMaxesFn()));
-
-      return results;
-    }
-  }
-
-  /**
-   * Options supported by {@link MaxPerKeyExamples}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Table to read from, specified as "
-        + "<project_id>:<dataset_id>.<table_id>")
-    @Default.String(WEATHER_SAMPLES_TABLE)
-    String getInput();
-    void setInput(String value);
-
-    @Description("Table to write to, specified as "
-        + "<project_id>:<dataset_id>.<table_id>")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args)
-      throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-
-    // Build the table schema for the output table.
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("max_mean_temp").setType("FLOAT"));
-    TableSchema schema = new TableSchema().setFields(fields);
-
-    p.apply(BigQueryIO.Read.from(options.getInput()))
-     .apply(new MaxMeanTemp())
-     .apply(BigQueryIO.Write
-        .to(options.getOutput())
-        .withSchema(schema)
-        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/README.md b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/README.md
deleted file mode 100644
index 99f3080..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/README.md
+++ /dev/null
@@ -1,55 +0,0 @@
-
-# "Cookbook" Examples
-
-This directory holds simple "cookbook" examples, which show how to define
-commonly-used data analysis patterns that you would likely incorporate into a
-larger Dataflow pipeline. They include:
-
- <ul>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java">BigQueryTornadoes</a>
-  &mdash; An example that reads the public samples of weather data from Google
-  BigQuery, counts the number of tornadoes that occur in each month, and
-  writes the results to BigQuery. Demonstrates reading/writing BigQuery,
-  counting a <code>PCollection</code>, and user-defined <code>PTransforms</code>.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java">CombinePerKeyExamples</a>
-  &mdash; An example that reads the public &quot;Shakespeare&quot; data, and for
-  each word in the dataset that exceeds a given length, generates a string
-  containing the list of play names in which that word appears.
-  Demonstrates the <code>Combine.perKey</code>
-  transform, which lets you combine the values in a key-grouped
-  <code>PCollection</code>.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java">DatastoreWordCount</a>
-  &mdash; An example that shows you how to read from Google Cloud Datastore.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java">DeDupExample</a>
-  &mdash; An example that uses Shakespeare's plays as plain text files, and
-  removes duplicate lines across all the files. Demonstrates the
-  <code>RemoveDuplicates</code>, <code>TextIO.Read</code>,
-  and <code>TextIO.Write</code> transforms, and how to wire transforms together.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java">FilterExamples</a>
-  &mdash; An example that shows different approaches to filtering, including
-  selection and projection. It also shows how to dynamically set parameters
-  by defining and using new pipeline options, and use how to use a value derived
-  by a pipeline. Demonstrates the <code>Mean</code> transform,
-  <code>Options</code> configuration, and using pipeline-derived data as a side
-  input.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java">JoinExamples</a>
-  &mdash; An example that shows how to join two collections. It uses a
-  sample of the <a href="http://goo.gl/OB6oin">GDELT &quot;world event&quot;
-  data</a>, joining the event <code>action</code> country code against a table
-  that maps country codes to country names. Demonstrates the <code>Join</code>
-  operation, and using multiple input sources.
-  </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java">MaxPerKeyExamples</a>
-  &mdash; An example that reads the public samples of weather data from BigQuery,
-  and finds the maximum temperature (<code>mean_temp</code>) for each month.
-  Demonstrates the <code>Max</code> statistical combination transform, and how to
-  find the max-per-key group.
-  </li>
-  </ul>
-
-See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
-README](../../../../../../../../../README.md) for
-information about how to run these examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/TriggerExample.java
deleted file mode 100644
index e32596d..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/TriggerExample.java
+++ /dev/null
@@ -1,565 +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 com.google.cloud.dataflow.examples.cookbook;
-
-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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.examples.common.PubsubFileInjector;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * This example illustrates the basic concepts behind triggering. It shows how to use different
- * trigger definitions to produce partial (speculative) results before all the data is processed and
- * to control when updated results are produced for late data. The example performs a streaming
- * analysis of the data coming in from PubSub and writes the results to BigQuery. It divides the
- * data into {@link Window windows} to be processed, and demonstrates using various kinds of {@link
- * Trigger triggers} to control when the results for each window are emitted.
- *
- * <p> This example uses a portion of real traffic data from San Diego freeways. It contains
- * readings from sensor stations set up along each freeway. Each sensor reading includes a
- * calculation of the 'total flow' across all lanes in that freeway direction.
- *
- * <p> Concepts:
- * <pre>
- *   1. The default triggering behavior
- *   2. Late data with the default trigger
- *   3. How to get speculative estimates
- *   4. Combining late data and speculative estimates
- * </pre>
- *
- * <p> Before running this example, it will be useful to familiarize yourself with Dataflow triggers
- * and understand the concept of 'late data',
- * See:  <a href="https://cloud.google.com/dataflow/model/triggers">
- * https://cloud.google.com/dataflow/model/triggers </a> and
- * <a href="https://cloud.google.com/dataflow/model/windowing#Advanced">
- * https://cloud.google.com/dataflow/model/windowing#Advanced </a>
- *
- * <p> The example pipeline reads data from a Pub/Sub topic. By default, running the example will
- * also run an auxiliary pipeline to inject data from the default {@code --input} file to the
- * {@code --pubsubTopic}. The auxiliary pipeline puts a timestamp on the injected data so that the
- * example pipeline can operate on <i>event time</i> (rather than arrival time). The auxiliary
- * pipeline also randomly simulates late data, by setting the timestamps of some of the data
- * elements to be in the past. You may override the default {@code --input} with the file of your
- * choosing or set {@code --input=""} which will disable the automatic Pub/Sub injection, and allow
- * you to use a separate tool to publish to the given topic.
- *
- * <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 {@code --pubsubTopic}, {@code --bigQueryDataset}, and
- * {@code --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
- * the example will try to create them.
- *
- * <p> The pipeline outputs its results to a BigQuery table.
- * Here are some queries you can use to see interesting results:
- * Replace {@code <enter_table_name>} in the query below with the name of the BigQuery table.
- * Replace {@code <enter_window_interval>} in the query below with the window interval.
- *
- * <p> To see the results of the default trigger,
- * Note: When you start up your pipeline, you'll initially see results from 'late' data. Wait after
- * the window duration, until the first pane of non-late data has been emitted, to see more
- * interesting results.
- * {@code SELECT * FROM enter_table_name WHERE trigger_type = "default" ORDER BY window DESC}
- *
- * <p> To see the late data i.e. dropped by the default trigger,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "withAllowedLateness" and
- * (timing = "LATE" or timing = "ON_TIME") and freeway = "5" ORDER BY window DESC, processing_time}
- *
- * <p>To see the the difference between accumulation mode and discarding mode,
- * {@code SELECT * FROM <enter_table_name> WHERE (timing = "LATE" or timing = "ON_TIME") AND
- * (trigger_type = "withAllowedLateness" or trigger_type = "sequential") and freeway = "5" ORDER BY
- * window DESC, processing_time}
- *
- * <p> To see speculative results every minute,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "speculative" and freeway = "5"
- * ORDER BY window DESC, processing_time}
- *
- * <p> To see speculative results every five minutes after the end of the window
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "sequential" and timing != "EARLY"
- * and freeway = "5" ORDER BY window DESC, processing_time}
- *
- * <p> To see the first and the last pane for a freeway in a window for all the trigger types,
- * {@code SELECT * FROM <enter_table_name> WHERE (isFirst = true or isLast = true) ORDER BY window}
- *
- * <p> To reduce the number of results for each query we can add additional where clauses.
- * For examples, To see the results of the default trigger,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "default" AND freeway = "5" AND
- * window = "<enter_window_interval>"}
- *
- * <p> The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
- * and then exits.
- */
-
-public class TriggerExample {
-  //Numeric value of fixed window duration, in minutes
-  public static final int WINDOW_DURATION = 30;
-  // Constants used in triggers.
-  // Speeding up ONE_MINUTE or FIVE_MINUTES helps you get an early approximation of results.
-  // ONE_MINUTE is used only with processing time before the end of the window
-  public static final Duration ONE_MINUTE = Duration.standardMinutes(1);
-  // FIVE_MINUTES is used only with processing time after the end of the window
-  public static final Duration FIVE_MINUTES = Duration.standardMinutes(5);
-  // ONE_DAY is used to specify the amount of lateness allowed for the data elements.
-  public static final Duration ONE_DAY = Duration.standardDays(1);
-
-  /**
-   * This transform demonstrates using triggers to control when data is produced for each window
-   * Consider an example to understand the results generated by each type of trigger.
-   * The example uses "freeway" as the key. Event time is the timestamp associated with the data
-   * element and processing time is the time when the data element gets processed in the pipeline.
-   * For freeway 5, suppose there are 10 elements in the [10:00:00, 10:30:00) window.
-   * Key (freeway) | Value (total_flow) | event time | processing time
-   * 5             | 50                 | 10:00:03   | 10:00:47
-   * 5             | 30                 | 10:01:00   | 10:01:03
-   * 5             | 30                 | 10:02:00   | 11:07:00
-   * 5             | 20                 | 10:04:10   | 10:05:15
-   * 5             | 60                 | 10:05:00   | 11:03:00
-   * 5             | 20                 | 10:05:01   | 11.07:30
-   * 5             | 60                 | 10:15:00   | 10:27:15
-   * 5             | 40                 | 10:26:40   | 10:26:43
-   * 5             | 60                 | 10:27:20   | 10:27:25
-   * 5             | 60                 | 10:29:00   | 11:11:00
-   *
-   * <p> Dataflow tracks a watermark which records up to what point in event time the data is
-   * complete. For the purposes of the example, we'll assume the watermark is approximately 15m
-   * behind the current processing time. In practice, the actual value would vary over time based
-   * on the systems knowledge of the current PubSub delay and contents of the backlog (data
-   * that has not yet been processed).
-   *
-   * <p> If the watermark is 15m behind, then the window [10:00:00, 10:30:00) (in event time) would
-   * close at 10:44:59, when the watermark passes 10:30:00.
-   */
-  static class CalculateTotalFlow
-  extends PTransform <PCollection<KV<String, Integer>>, PCollectionList<TableRow>> {
-    private int windowDuration;
-
-    CalculateTotalFlow(int windowDuration) {
-      this.windowDuration = windowDuration;
-    }
-
-    @Override
-    public PCollectionList<TableRow> apply(PCollection<KV<String, Integer>> flowInfo) {
-
-      // Concept #1: The default triggering behavior
-      // By default Dataflow uses a trigger which fires when the watermark has passed the end of the
-      // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
-
-      // The system also defaults to dropping late data -- data which arrives after the watermark
-      // has passed the event timestamp of the arriving element. This means that the default trigger
-      // will only fire once.
-
-      // Each pane produced by the default trigger with no allowed lateness will be the first and
-      // last pane in the window, and will be ON_TIME.
-
-      // The results for the example above with the default trigger and zero allowed lateness
-      // would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
-      // 5             | 260                | 6                 | true    | true   | ON_TIME
-
-      // At 11:03:00 (processing time) the system watermark may have advanced to 10:54:00. As a
-      // result, when the data record with event time 10:05:00 arrives at 11:03:00, it is considered
-      // late, and dropped.
-
-      PCollection<TableRow> defaultTriggerResults = flowInfo
-          .apply("Default", Window
-              // The default window duration values work well if you're running the default input
-              // file. You may want to adjust the window duration otherwise.
-              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
-              // The default trigger first emits output when the system's watermark passes the end
-              // of the window.
-              .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
-              // Late data is dropped
-              .withAllowedLateness(Duration.ZERO)
-              // Discard elements after emitting each pane.
-              // With no allowed lateness and the specified trigger there will only be a single
-              // pane, so this doesn't have a noticeable effect. See concept 2 for more details.
-              .discardingFiredPanes())
-          .apply(new TotalFlow("default"));
-
-      // Concept #2: Late data with the default trigger
-      // This uses the same trigger as concept #1, but allows data that is up to ONE_DAY late. This
-      // leads to each window staying open for ONE_DAY after the watermark has passed the end of the
-      // window. Any late data will result in an additional pane being fired for that same window.
-
-      // The first pane produced will be ON_TIME and the remaining panes will be LATE.
-      // To definitely get the last pane when the window closes, use
-      // .withAllowedLateness(ONE_DAY, ClosingBehavior.FIRE_ALWAYS).
-
-      // The results for the example above with the default trigger and ONE_DAY allowed lateness
-      // would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
-      // 5             | 260                | 6                 | true    | false  | ON_TIME
-      // 5             | 60                 | 1                 | false   | false  | LATE
-      // 5             | 30                 | 1                 | false   | false  | LATE
-      // 5             | 20                 | 1                 | false   | false  | LATE
-      // 5             | 60                 | 1                 | false   | false  | LATE
-      PCollection<TableRow> withAllowedLatenessResults = flowInfo
-          .apply("WithLateData", Window
-              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
-              // Late data is emitted as it arrives
-              .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
-              // Once the output is produced, the pane is dropped and we start preparing the next
-              // pane for the window
-              .discardingFiredPanes()
-              // Late data is handled up to one day
-              .withAllowedLateness(ONE_DAY))
-          .apply(new TotalFlow("withAllowedLateness"));
-
-      // Concept #3: How to get speculative estimates
-      // We can specify a trigger that fires independent of the watermark, for instance after
-      // ONE_MINUTE of processing time. This allows us to produce speculative estimates before
-      // all the data is available. Since we don't have any triggers that depend on the watermark
-      // we don't get an ON_TIME firing. Instead, all panes are either EARLY or LATE.
-
-      // We also use accumulatingFiredPanes to build up the results across each pane firing.
-
-      // The results for the example above for this trigger would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
-      // 5             | 80                 | 2                 | true    | false  | EARLY
-      // 5             | 100                | 3                 | false   | false  | EARLY
-      // 5             | 260                | 6                 | false   | false  | EARLY
-      // 5             | 320                | 7                 | false   | false  | LATE
-      // 5             | 370                | 9                 | false   | false  | LATE
-      // 5             | 430                | 10                | false   | false  | LATE
-      PCollection<TableRow> speculativeResults = flowInfo
-          .apply("Speculative" , Window
-              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
-              // Trigger fires every minute.
-              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
-                  // Speculative every ONE_MINUTE
-                  .plusDelayOf(ONE_MINUTE)))
-              // After emitting each pane, it will continue accumulating the elements so that each
-              // approximation includes all of the previous data in addition to the newly arrived
-              // data.
-              .accumulatingFiredPanes()
-              .withAllowedLateness(ONE_DAY))
-          .apply(new TotalFlow("speculative"));
-
-      // Concept #4: Combining late data and speculative estimates
-      // We can put the previous concepts together to get EARLY estimates, an ON_TIME result,
-      // and LATE updates based on late data.
-
-      // Each time a triggering condition is satisfied it advances to the next trigger.
-      // If there are new elements this trigger emits a window under following condition:
-      // > Early approximations every minute till the end of the window.
-      // > An on-time firing when the watermark has passed the end of the window
-      // > Every five minutes of late data.
-
-      // Every pane produced will either be EARLY, ON_TIME or LATE.
-
-      // The results for the example above for this trigger would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
-      // 5             | 80                 | 2                 | true    | false  | EARLY
-      // 5             | 100                | 3                 | false   | false  | EARLY
-      // 5             | 260                | 6                 | false   | false  | EARLY
-      // [First pane fired after the end of the window]
-      // 5             | 320                | 7                 | false   | false  | ON_TIME
-      // 5             | 430                | 10                | false   | false  | LATE
-
-      // For more possibilities of how to build advanced triggers, see {@link Trigger}.
-      PCollection<TableRow> sequentialResults = flowInfo
-          .apply("Sequential", Window
-              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
-              .triggering(AfterEach.inOrder(
-                  Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
-                      // Speculative every ONE_MINUTE
-                      .plusDelayOf(ONE_MINUTE)).orFinally(AfterWatermark.pastEndOfWindow()),
-                  Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
-                      // Late data every FIVE_MINUTES
-                      .plusDelayOf(FIVE_MINUTES))))
-              .accumulatingFiredPanes()
-              // For up to ONE_DAY
-              .withAllowedLateness(ONE_DAY))
-          .apply(new TotalFlow("sequential"));
-
-      // Adds the results generated by each trigger type to a PCollectionList.
-      PCollectionList<TableRow> resultsList = PCollectionList.of(defaultTriggerResults)
-          .and(withAllowedLatenessResults)
-          .and(speculativeResults)
-          .and(sequentialResults);
-
-      return resultsList;
-    }
-  }
-
-  //////////////////////////////////////////////////////////////////////////////////////////////////
-  // The remaining parts of the pipeline are needed to produce the output for each
-  // concept above. Not directly relevant to understanding the trigger examples.
-
-  /**
-   * Calculate total flow and number of records for each freeway and format the results to TableRow
-   * objects, to save to BigQuery.
-   */
-  static class TotalFlow extends
-  PTransform <PCollection<KV<String, Integer>>, PCollection<TableRow>> {
-    private String triggerType;
-
-    public TotalFlow(String triggerType) {
-      this.triggerType = triggerType;
-    }
-
-    @Override
-    public PCollection<TableRow> apply(PCollection<KV<String, Integer>> flowInfo) {
-      PCollection<KV<String, Iterable<Integer>>> flowPerFreeway = flowInfo
-          .apply(GroupByKey.<String, Integer>create());
-
-      PCollection<KV<String, String>> results = flowPerFreeway.apply(ParDo.of(
-          new DoFn <KV<String, Iterable<Integer>>, KV<String, String>>() {
-
-            @Override
-            public void processElement(ProcessContext c) throws Exception {
-              Iterable<Integer> flows = c.element().getValue();
-              Integer sum = 0;
-              Long numberOfRecords = 0L;
-              for (Integer value : flows) {
-                sum += value;
-                numberOfRecords++;
-              }
-              c.output(KV.of(c.element().getKey(), sum + "," + numberOfRecords));
-            }
-          }));
-      PCollection<TableRow> output = results.apply(ParDo.of(new FormatTotalFlow(triggerType)));
-      return output;
-    }
-  }
-
-  /**
-   * Format the results of the Total flow calculation to a TableRow, to save to BigQuery.
-   * Adds the triggerType, pane information, processing time and the window timestamp.
-   * */
-  static class FormatTotalFlow extends DoFn<KV<String, String>, TableRow>
-  implements  RequiresWindowAccess {
-    private String triggerType;
-
-    public FormatTotalFlow(String triggerType) {
-      this.triggerType = triggerType;
-    }
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      String[] values = c.element().getValue().split(",");
-      TableRow row = new TableRow()
-          .set("trigger_type", triggerType)
-          .set("freeway", c.element().getKey())
-          .set("total_flow", Integer.parseInt(values[0]))
-          .set("number_of_records", Long.parseLong(values[1]))
-          .set("window", c.window().toString())
-          .set("isFirst", c.pane().isFirst())
-          .set("isLast", c.pane().isLast())
-          .set("timing", c.pane().getTiming().toString())
-          .set("event_time", c.timestamp().toString())
-          .set("processing_time", Instant.now().toString());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Extract the freeway and total flow in a reading.
-   * Freeway is used as key since we are calculating the total flow for each freeway.
-   */
-  static class ExtractFlowInfo extends DoFn<String, KV<String, Integer>> {
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      String[] laneInfo = c.element().split(",");
-      if (laneInfo[0].equals("timestamp")) {
-        // Header row
-        return;
-      }
-      if (laneInfo.length < 48) {
-        //Skip the invalid input.
-        return;
-      }
-      String freeway = laneInfo[2];
-      Integer totalFlow = tryIntegerParse(laneInfo[7]);
-      // Ignore the records with total flow 0 to easily understand the working of triggers.
-      // Skip the records with total flow -1 since they are invalid input.
-      if (totalFlow == null || totalFlow <= 0) {
-        return;
-      }
-      c.output(KV.of(freeway,  totalFlow));
-    }
-  }
-
-  /**
-   * Inherits standard configuration options.
-   */
-  public interface TrafficFlowOptions
-      extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions, DataflowExampleOptions {
-
-    @Description("Input file to inject to Pub/Sub topic")
-    @Default.String("gs://dataflow-samples/traffic_sensor/"
-        + "Freeways-5Minaa2010-01-01_to_2010-02-15.csv")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Numeric value of window duration for fixed windows, in minutes")
-    @Default.Integer(WINDOW_DURATION)
-    Integer getWindowDuration();
-    void setWindowDuration(Integer value);
-  }
-
-  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
-
-  public static void main(String[] args) throws Exception {
-    TrafficFlowOptions options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(TrafficFlowOptions.class);
-    options.setStreaming(true);
-
-    // In order to cancel the pipelines automatically,
-    // {@code DataflowPipelineRunner} is forced to be used.
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setBigQuerySchema(getSchema());
-
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
-    dataflowUtils.setup();
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    TableReference tableRef = getTableReference(options.getProject(),
-        options.getBigQueryDataset(), options.getBigQueryTable());
-
-    PCollectionList<TableRow> resultList = pipeline.apply(PubsubIO.Read.named("ReadPubsubInput")
-        .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
-        .topic(options.getPubsubTopic()))
-        .apply(ParDo.of(new ExtractFlowInfo()))
-        .apply(new CalculateTotalFlow(options.getWindowDuration()));
-
-    for (int i = 0; i < resultList.size(); i++){
-      resultList.get(i).apply(BigQueryIO.Write.to(tableRef).withSchema(getSchema()));
-    }
-
-    PipelineResult result = pipeline.run();
-    if (!options.getInput().isEmpty()){
-      //Inject the data into the pubsub topic
-      dataflowUtils.runInjectorPipeline(runInjector(options));
-    }
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exits.
-    dataflowUtils.waitToFinish(result);
-  }
-
-  private static Pipeline runInjector(TrafficFlowOptions options){
-    DataflowPipelineOptions copiedOptions = options.cloneAs(DataflowPipelineOptions.class);
-    copiedOptions.setStreaming(false);
-    copiedOptions.setNumWorkers(options.as(DataflowExampleOptions.class).getInjectorNumWorkers());
-    copiedOptions.setJobName(options.getJobName() + "-injector");
-    Pipeline injectorPipeline = Pipeline.create(copiedOptions);
-    injectorPipeline
-    .apply(TextIO.Read.named("ReadMyFile").from(options.getInput()))
-    .apply(ParDo.named("InsertRandomDelays").of(new InsertDelays()))
-    .apply(IntraBundleParallelization.of(PubsubFileInjector
-        .withTimestampLabelKey(PUBSUB_TIMESTAMP_LABEL_KEY)
-        .publish(options.getPubsubTopic()))
-        .withMaxParallelism(20));
-
-    return injectorPipeline;
-  }
-
-  /**
-   * Add current time to each record.
-   * Also insert a delay at random to demo the triggers.
-   */
-  public static class InsertDelays extends DoFn<String, String> {
-    private static final double THRESHOLD = 0.001;
-    // MIN_DELAY and MAX_DELAY in minutes.
-    private static final int MIN_DELAY = 1;
-    private static final int MAX_DELAY = 100;
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      Instant timestamp = Instant.now();
-      if (Math.random() < THRESHOLD){
-        int range = MAX_DELAY - MIN_DELAY;
-        int delayInMinutes = (int) (Math.random() * range) + MIN_DELAY;
-        long delayInMillis = TimeUnit.MINUTES.toMillis(delayInMinutes);
-        timestamp = new Instant(timestamp.getMillis() - delayInMillis);
-      }
-      c.outputWithTimestamp(c.element(), timestamp);
-    }
-  }
-
-
-  /**Sets the table reference. **/
-  private static TableReference getTableReference(String project, String dataset, String table){
-    TableReference tableRef = new TableReference();
-    tableRef.setProjectId(project);
-    tableRef.setDatasetId(dataset);
-    tableRef.setTableId(table);
-    return tableRef;
-  }
-
-  /** Defines the BigQuery schema used for the output. */
-  private static TableSchema getSchema() {
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("trigger_type").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("freeway").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("total_flow").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("number_of_records").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("window").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("isFirst").setType("BOOLEAN"));
-    fields.add(new TableFieldSchema().setName("isLast").setType("BOOLEAN"));
-    fields.add(new TableFieldSchema().setName("timing").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("event_time").setType("TIMESTAMP"));
-    fields.add(new TableFieldSchema().setName("processing_time").setType("TIMESTAMP"));
-    TableSchema schema = new TableSchema().setFields(fields);
-    return schema;
-  }
-
-  private static Integer tryIntegerParse(String number) {
-    try {
-      return Integer.parseInt(number);
-    } catch (NumberFormatException e) {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..7134bca
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
@@ -0,0 +1,199 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Pattern;
+
+
+/**
+ * An example that verifies word counts in Shakespeare and includes Dataflow best practices.
+ *
+ * <p>This class, {@link DebuggingWordCount}, is the third in a series of four successively more
+ * detailed 'word count' examples. You may first want to take a look at {@link MinimalWordCount}
+ * and {@link WordCount}. After you've looked at this example, then see the
+ * {@link WindowedWordCount} pipeline, for introduction of additional concepts.
+ *
+ * <p>Basic concepts, also in the MinimalWordCount and WordCount examples:
+ * Reading text files; counting a PCollection; executing a Pipeline both locally
+ * and using the Dataflow service; defining DoFns.
+ *
+ * <p>New Concepts:
+ * <pre>
+ *   1. Logging to Cloud Logging
+ *   2. Controlling Dataflow worker log levels
+ *   3. Creating a custom aggregator
+ *   4. Testing your Pipeline via PAssert
+ * </pre>
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service and the additional logging discussed
+ * below, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ * }
+ * </pre>
+ *
+ * <p>Note that when you run via <code>mvn exec</code>, you may need to escape
+ * the quotations as appropriate for your shell. For example, in <code>bash</code>:
+ * <pre>
+ * mvn compile exec:java ... \
+ *   -Dexec.args="... \
+ *     --workerLogLevelOverrides={\\\"com.google.cloud.dataflow.examples\\\":\\\"DEBUG\\\"}"
+ * </pre>
+ *
+ * <p>Concept #2: Dataflow workers which execute user code are configured to log to Cloud
+ * Logging by default at "INFO" log level and higher. One may override log levels for specific
+ * logging namespaces by specifying:
+ * <pre><code>
+ *   --workerLogLevelOverrides={"Name1":"Level1","Name2":"Level2",...}
+ * </code></pre>
+ * For example, by specifying:
+ * <pre><code>
+ *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ * </code></pre>
+ * when executing this pipeline using the Dataflow service, Cloud Logging would contain only
+ * "DEBUG" or higher level logs for the {@code com.google.cloud.dataflow.examples} package in
+ * addition to the default "INFO" or higher level logs. In addition, the default Dataflow worker
+ * logging configuration can be overridden by specifying
+ * {@code --defaultWorkerLogLevel=<one of TRACE, DEBUG, INFO, WARN, ERROR>}. For example,
+ * by specifying {@code --defaultWorkerLogLevel=DEBUG} when executing this pipeline with
+ * the Dataflow service, Cloud Logging would contain all "DEBUG" or higher level logs. Note
+ * that changing the default worker log level to TRACE or DEBUG will significantly increase
+ * the amount of logs output.
+ *
+ * <p>The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be
+ * overridden with {@code --inputFile}.
+ */
+public class DebuggingWordCount {
+  /** A DoFn that filters for a specific key based upon a regular expression. */
+  public static class FilterTextFn extends DoFn<KV<String, Long>, KV<String, Long>> {
+    /**
+     * Concept #1: The logger below uses the fully qualified class name of FilterTextFn
+     * as the logger. All log statements emitted by this logger will be referenced by this name
+     * and will be visible in the Cloud Logging UI. Learn more at https://cloud.google.com/logging
+     * about the Cloud Logging UI.
+     */
+    private static final Logger LOG = LoggerFactory.getLogger(FilterTextFn.class);
+
+    private final Pattern filter;
+    public FilterTextFn(String pattern) {
+      filter = Pattern.compile(pattern);
+    }
+
+    /**
+     * Concept #3: A custom aggregator can track values in your pipeline as it runs. Those
+     * values will be displayed in the Dataflow Monitoring UI when this pipeline is run using the
+     * Dataflow service. These aggregators below track the number of matched and unmatched words.
+     * Learn more at https://cloud.google.com/dataflow/pipelines/dataflow-monitoring-intf about
+     * the Dataflow Monitoring UI.
+     */
+    private final Aggregator<Long, Long> matchedWords =
+        createAggregator("matchedWords", new Sum.SumLongFn());
+    private final Aggregator<Long, Long> unmatchedWords =
+        createAggregator("umatchedWords", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      if (filter.matcher(c.element().getKey()).matches()) {
+        // Log at the "DEBUG" level each element that we match. When executing this pipeline
+        // using the Dataflow service, these log lines will appear in the Cloud Logging UI
+        // only if the log level is set to "DEBUG" or lower.
+        LOG.debug("Matched: " + c.element().getKey());
+        matchedWords.addValue(1L);
+        c.output(c.element());
+      } else {
+        // Log at the "TRACE" level each element that is not matched. Different log levels
+        // can be used to control the verbosity of logging providing an effective mechanism
+        // to filter less important information.
+        LOG.trace("Did not match: " + c.element().getKey());
+        unmatchedWords.addValue(1L);
+      }
+    }
+  }
+
+  /**
+   * Options supported by {@link DebuggingWordCount}.
+   *
+   * <p>Inherits standard configuration options and all options defined in
+   * {@link WordCount.WordCountOptions}.
+   */
+  public static interface WordCountOptions extends WordCount.WordCountOptions {
+
+    @Description("Regex filter pattern to use in DebuggingWordCount. "
+        + "Only words matching this pattern will be counted.")
+    @Default.String("Flourish|stomach")
+    String getFilterPattern();
+    void setFilterPattern(String value);
+  }
+
+  public static void main(String[] args) {
+    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
+      .as(WordCountOptions.class);
+    Pipeline p = Pipeline.create(options);
+
+    PCollection<KV<String, Long>> filteredWords =
+        p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
+         .apply(new WordCount.CountWords())
+         .apply(ParDo.of(new FilterTextFn(options.getFilterPattern())));
+
+    /**
+     * Concept #4: PAssert is a set of convenient PTransforms in the style of
+     * Hamcrest's collection matchers that can be used when writing Pipeline level tests
+     * to validate the contents of PCollections. PAssert is best used in unit tests
+     * with small data sets but is demonstrated here as a teaching tool.
+     *
+     * <p>Below we verify that the set of filtered words matches our expected counts. Note
+     * that PAssert does not provide any output and that successful completion of the
+     * Pipeline implies that the expectations were met. Learn more at
+     * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline on how to test
+     * your Pipeline and see {@link DebuggingWordCountTest} for an example unit test.
+     */
+    List<KV<String, Long>> expectedResults = Arrays.asList(
+        KV.of("Flourish", 3L),
+        KV.of("stomach", 1L));
+    PAssert.that(filteredWords).containsInAnyOrder(expectedResults);
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..f3be5a3
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -0,0 +1,118 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
+import com.google.cloud.dataflow.sdk.values.KV;
+
+
+/**
+ * An example that counts words in Shakespeare.
+ *
+ * <p>This class, {@link MinimalWordCount}, is the first in a series of four successively more
+ * detailed 'word count' examples. Here, for simplicity, we don't show any error-checking or
+ * argument processing, and focus on construction of the pipeline, which chains together the
+ * application of core transforms.
+ *
+ * <p>Next, see the {@link WordCount} pipeline, then the {@link DebuggingWordCount}, and finally
+ * the {@link WindowedWordCount} pipeline, for more detailed examples that introduce additional
+ * concepts.
+ *
+ * <p>Concepts:
+ * <pre>
+ *   1. Reading data from text files
+ *   2. Specifying 'inline' transforms
+ *   3. Counting a PCollection
+ *   4. Writing data to Cloud Storage as text files
+ * </pre>
+ *
+ * <p>To execute this pipeline, first edit the code to set your project ID, the staging
+ * location, and the output location. The specified GCS bucket(s) must already exist.
+ *
+ * <p>Then, run the pipeline as described in the README. It will be deployed and run using the
+ * Dataflow service. No args are required to run the pipeline. You can see the results in your
+ * output bucket in the GCS browser.
+ */
+public class MinimalWordCount {
+
+  public static void main(String[] args) {
+    // Create a DataflowPipelineOptions object. This object lets us set various execution
+    // options for our pipeline, such as the associated Cloud Platform project and the location
+    // in Google Cloud Storage to stage files.
+    DataflowPipelineOptions options = PipelineOptionsFactory.create()
+      .as(DataflowPipelineOptions.class);
+    options.setRunner(BlockingDataflowPipelineRunner.class);
+    // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud.
+    options.setProject("SET_YOUR_PROJECT_ID_HERE");
+    // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files.
+    options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY");
+
+    // Create the Pipeline object with the options we defined above.
+    Pipeline p = Pipeline.create(options);
+
+    // Apply the pipeline's transforms.
+
+    // Concept #1: Apply a root transform to the pipeline; in this case, TextIO.Read to read a set
+    // of input text files. TextIO.Read returns a PCollection where each element is one line from
+    // the input text (a set of Shakespeare's texts).
+    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
+     // Concept #2: Apply a ParDo transform to our PCollection of text lines. This ParDo invokes a
+     // DoFn (defined in-line) on each element that tokenizes the text line into individual words.
+     // The ParDo returns a PCollection<String>, where each element is an individual word in
+     // Shakespeare's collected texts.
+     .apply(ParDo.named("ExtractWords").of(new DoFn<String, String>() {
+                       @Override
+                       public void processElement(ProcessContext c) {
+                         for (String word : c.element().split("[^a-zA-Z']+")) {
+                           if (!word.isEmpty()) {
+                             c.output(word);
+                           }
+                         }
+                       }
+                     }))
+     // Concept #3: Apply the Count transform to our PCollection of individual words. The Count
+     // transform returns a new PCollection of key/value pairs, where each key represents a unique
+     // word in the text. The associated value is the occurrence count for that word.
+     .apply(Count.<String>perElement())
+     // Apply a MapElements transform that formats our PCollection of word counts into a printable
+     // string, suitable for writing to an output file.
+     .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
+                       @Override
+                       public String apply(KV<String, Long> input) {
+                         return input.getKey() + ": " + input.getValue();
+                       }
+                     }))
+     // Concept #4: Apply a write transform, TextIO.Write, at the end of the pipeline.
+     // TextIO.Write writes the contents of a PCollection (in this case, our PCollection of
+     // formatted strings) to a series of text files in Google Cloud Storage.
+     // CHANGE 3/3: The Google Cloud Storage path is required for outputting the results to.
+     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
+
+    // Run the pipeline.
+    p.run();
+  }
+}


[74/74] incubator-beam git commit: This closes #176

Posted by da...@apache.org.
This closes #176


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

Branch: refs/heads/master
Commit: 96765f19b1bd8149240cd77eb7cf7fb636e477e4
Parents: c4cbbb1 75a1905
Author: Davor Bonaci <da...@google.com>
Authored: Wed Apr 13 21:31:08 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Wed Apr 13 21:31:08 2016 -0700

----------------------------------------------------------------------
 contrib/hadoop/pom.xml                          |    8 +-
 .../contrib/hadoop/HadoopFileSource.java        |  486 ---
 .../dataflow/contrib/hadoop/WritableCoder.java  |  111 -
 .../apache/contrib/hadoop/HadoopFileSource.java |  486 +++
 .../apache/contrib/hadoop/WritableCoder.java    |  111 +
 .../contrib/hadoop/HadoopFileSourceTest.java    |  190 --
 .../contrib/hadoop/WritableCoderTest.java       |   37 -
 .../contrib/hadoop/HadoopFileSourceTest.java    |  190 ++
 .../contrib/hadoop/WritableCoderTest.java       |   37 +
 contrib/join-library/pom.xml                    |    8 +-
 .../dataflow/contrib/joinlibrary/Join.java      |  186 --
 .../org/apache/contrib/joinlibrary/Join.java    |  186 ++
 .../contrib/joinlibrary/InnerJoinTest.java      |  143 -
 .../contrib/joinlibrary/OuterLeftJoinTest.java  |  153 -
 .../contrib/joinlibrary/OuterRightJoinTest.java |  153 -
 .../contrib/joinlibrary/InnerJoinTest.java      |  143 +
 .../contrib/joinlibrary/OuterLeftJoinTest.java  |  153 +
 .../contrib/joinlibrary/OuterRightJoinTest.java |  153 +
 examples/java/pom.xml                           |    4 +-
 .../dataflow/examples/DebuggingWordCount.java   |  199 --
 .../dataflow/examples/MinimalWordCount.java     |  118 -
 .../dataflow/examples/WindowedWordCount.java    |  270 --
 .../cloud/dataflow/examples/WordCount.java      |  207 --
 .../examples/common/DataflowExampleOptions.java |   37 -
 .../examples/common/DataflowExampleUtils.java   |  488 ---
 .../common/ExampleBigQueryTableOptions.java     |   56 -
 ...xamplePubsubTopicAndSubscriptionOptions.java |   47 -
 .../common/ExamplePubsubTopicOptions.java       |   47 -
 .../examples/common/PubsubFileInjector.java     |  154 -
 .../examples/complete/AutoComplete.java         |  517 ---
 .../cloud/dataflow/examples/complete/README.md  |   44 -
 .../examples/complete/StreamingWordExtract.java |  164 -
 .../cloud/dataflow/examples/complete/TfIdf.java |  432 ---
 .../examples/complete/TopWikipediaSessions.java |  224 --
 .../examples/complete/TrafficMaxLaneFlow.java   |  426 ---
 .../examples/complete/TrafficRoutes.java        |  460 ---
 .../examples/cookbook/BigQueryTornadoes.java    |  180 --
 .../cookbook/CombinePerKeyExamples.java         |  224 --
 .../examples/cookbook/DatastoreWordCount.java   |  270 --
 .../examples/cookbook/DeDupExample.java         |  101 -
 .../examples/cookbook/FilterExamples.java       |  267 --
 .../examples/cookbook/JoinExamples.java         |  186 --
 .../examples/cookbook/MaxPerKeyExamples.java    |  174 -
 .../cloud/dataflow/examples/cookbook/README.md  |   55 -
 .../examples/cookbook/TriggerExample.java       |  565 ----
 .../beam/examples/DebuggingWordCount.java       |  199 ++
 .../apache/beam/examples/MinimalWordCount.java  |  118 +
 .../apache/beam/examples/WindowedWordCount.java |  271 ++
 .../org/apache/beam/examples/WordCount.java     |  207 ++
 .../examples/common/DataflowExampleOptions.java |   37 +
 .../examples/common/DataflowExampleUtils.java   |  489 +++
 .../common/ExampleBigQueryTableOptions.java     |   57 +
 ...xamplePubsubTopicAndSubscriptionOptions.java |   47 +
 .../common/ExamplePubsubTopicOptions.java       |   47 +
 .../examples/common/PubsubFileInjector.java     |  155 +
 .../beam/examples/complete/AutoComplete.java    |  518 +++
 .../org/apache/beam/examples/complete/README.md |   44 +
 .../examples/complete/StreamingWordExtract.java |  165 +
 .../apache/beam/examples/complete/TfIdf.java    |  432 +++
 .../examples/complete/TopWikipediaSessions.java |  225 ++
 .../examples/complete/TrafficMaxLaneFlow.java   |  427 +++
 .../beam/examples/complete/TrafficRoutes.java   |  461 +++
 .../examples/cookbook/BigQueryTornadoes.java    |  181 ++
 .../cookbook/CombinePerKeyExamples.java         |  225 ++
 .../examples/cookbook/DatastoreWordCount.java   |  271 ++
 .../beam/examples/cookbook/DeDupExample.java    |  101 +
 .../beam/examples/cookbook/FilterExamples.java  |  268 ++
 .../beam/examples/cookbook/JoinExamples.java    |  187 ++
 .../examples/cookbook/MaxPerKeyExamples.java    |  175 +
 .../org/apache/beam/examples/cookbook/README.md |   55 +
 .../beam/examples/cookbook/TriggerExample.java  |  567 ++++
 .../examples/DebuggingWordCountTest.java        |   46 -
 .../cloud/dataflow/examples/WordCountTest.java  |   86 -
 .../examples/complete/AutoCompleteTest.java     |  182 --
 .../dataflow/examples/complete/TfIdfTest.java   |   68 -
 .../complete/TopWikipediaSessionsTest.java      |   63 -
 .../cookbook/BigQueryTornadoesTest.java         |   81 -
 .../cookbook/CombinePerKeyExamplesTest.java     |   91 -
 .../examples/cookbook/DeDupExampleTest.java     |   84 -
 .../examples/cookbook/FilterExamplesTest.java   |   86 -
 .../examples/cookbook/JoinExamplesTest.java     |  115 -
 .../cookbook/MaxPerKeyExamplesTest.java         |   86 -
 .../examples/cookbook/TriggerExampleTest.java   |  140 -
 .../beam/examples/DebuggingWordCountTest.java   |   45 +
 .../org/apache/beam/examples/WordCountTest.java |   86 +
 .../examples/complete/AutoCompleteTest.java     |  182 ++
 .../beam/examples/complete/TfIdfTest.java       |   68 +
 .../complete/TopWikipediaSessionsTest.java      |   64 +
 .../cookbook/BigQueryTornadoesTest.java         |   82 +
 .../cookbook/CombinePerKeyExamplesTest.java     |   92 +
 .../examples/cookbook/DeDupExampleTest.java     |   84 +
 .../examples/cookbook/FilterExamplesTest.java   |   87 +
 .../examples/cookbook/JoinExamplesTest.java     |  116 +
 .../cookbook/MaxPerKeyExamplesTest.java         |   87 +
 .../examples/cookbook/TriggerExampleTest.java   |  139 +
 .../examples/MinimalWordCountJava8.java         |   69 -
 .../examples/complete/game/GameStats.java       |  340 --
 .../examples/complete/game/HourlyTeamScore.java |  194 --
 .../examples/complete/game/LeaderBoard.java     |  238 --
 .../dataflow/examples/complete/game/README.md   |  113 -
 .../examples/complete/game/UserScore.java       |  240 --
 .../complete/game/injector/Injector.java        |  416 ---
 .../complete/game/injector/InjectorUtils.java   |  102 -
 .../injector/RetryHttpInitializerWrapper.java   |  129 -
 .../complete/game/utils/WriteToBigQuery.java    |  135 -
 .../game/utils/WriteWindowedToBigQuery.java     |   77 -
 .../beam/examples/MinimalWordCountJava8.java    |   69 +
 .../beam/examples/complete/game/GameStats.java  |  340 ++
 .../examples/complete/game/HourlyTeamScore.java |  193 ++
 .../examples/complete/game/LeaderBoard.java     |  238 ++
 .../beam/examples/complete/game/README.md       |  113 +
 .../beam/examples/complete/game/UserScore.java  |  240 ++
 .../complete/game/injector/Injector.java        |  416 +++
 .../complete/game/injector/InjectorUtils.java   |  101 +
 .../injector/RetryHttpInitializerWrapper.java   |  129 +
 .../complete/game/utils/WriteToBigQuery.java    |  136 +
 .../game/utils/WriteWindowedToBigQuery.java     |   78 +
 .../examples/MinimalWordCountJava8Test.java     |  104 -
 .../examples/complete/game/GameStatsTest.java   |   77 -
 .../complete/game/HourlyTeamScoreTest.java      |  112 -
 .../examples/complete/game/UserScoreTest.java   |  155 -
 .../examples/MinimalWordCountJava8Test.java     |  105 +
 .../examples/complete/game/GameStatsTest.java   |   77 +
 .../complete/game/HourlyTeamScoreTest.java      |  112 +
 .../examples/complete/game/UserScoreTest.java   |  155 +
 .../beam/runners/flink/examples/TFIDF.java      |   68 +-
 .../beam/runners/flink/examples/WordCount.java  |   18 +-
 .../flink/examples/streaming/AutoComplete.java  |   28 +-
 .../flink/examples/streaming/JoinExamples.java  |   30 +-
 .../KafkaWindowedWordCountExample.java          |   20 +-
 .../examples/streaming/WindowedWordCount.java   |   18 +-
 .../FlinkPipelineExecutionEnvironment.java      |    8 +-
 .../runners/flink/FlinkPipelineOptions.java     |   13 +-
 .../beam/runners/flink/FlinkPipelineRunner.java |   23 +-
 .../runners/flink/FlinkRunnerRegistrar.java     |   11 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |   12 +-
 .../apache/beam/runners/flink/io/ConsoleIO.java |    8 +-
 .../FlinkBatchPipelineTranslator.java           |   15 +-
 .../FlinkBatchTransformTranslators.java         |   58 +-
 .../FlinkBatchTranslationContext.java           |   21 +-
 .../translation/FlinkPipelineTranslator.java    |    4 +-
 .../FlinkStreamingPipelineTranslator.java       |   16 +-
 .../FlinkStreamingTransformTranslators.java     |   54 +-
 .../FlinkStreamingTranslationContext.java       |   14 +-
 .../FlinkCoGroupKeyedListAggregator.java        |   11 +-
 .../functions/FlinkCreateFunction.java          |    5 +-
 .../functions/FlinkDoFnFunction.java            |   35 +-
 .../FlinkKeyedListAggregationFunction.java      |    7 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   33 +-
 .../FlinkMultiOutputPruningFunction.java        |    3 +-
 .../functions/FlinkPartialReduceFunction.java   |    9 +-
 .../functions/FlinkReduceFunction.java          |   10 +-
 .../flink/translation/functions/UnionCoder.java |   15 +-
 .../translation/types/CoderComparator.java      |    5 +-
 .../translation/types/CoderTypeInformation.java |   10 +-
 .../translation/types/CoderTypeSerializer.java  |    7 +-
 .../translation/types/KvCoderComperator.java    |    9 +-
 .../types/KvCoderTypeInformation.java           |   10 +-
 .../types/VoidCoderTypeSerializer.java          |    2 +-
 .../wrappers/CombineFnAggregatorWrapper.java    |   10 +-
 .../wrappers/DataInputViewWrapper.java          |    2 +-
 .../wrappers/DataOutputViewWrapper.java         |    2 +-
 .../SerializableFnAggregatorWrapper.java        |   10 +-
 .../translation/wrappers/SinkOutputFormat.java  |   11 +-
 .../translation/wrappers/SourceInputFormat.java |   11 +-
 .../translation/wrappers/SourceInputSplit.java  |    3 +-
 .../streaming/FlinkAbstractParDoWrapper.java    |   30 +-
 .../FlinkGroupAlsoByWindowWrapper.java          |   75 +-
 .../streaming/FlinkGroupByKeyWrapper.java       |   11 +-
 .../streaming/FlinkParDoBoundMultiWrapper.java  |   18 +-
 .../streaming/FlinkParDoBoundWrapper.java       |   26 +-
 .../io/FlinkStreamingCreateFunction.java        |   10 +-
 .../streaming/io/UnboundedFlinkSource.java      |   12 +-
 .../streaming/io/UnboundedSocketSource.java     |   14 +-
 .../streaming/io/UnboundedSourceWrapper.java    |   18 +-
 .../state/AbstractFlinkTimerInternals.java      |   17 +-
 .../streaming/state/FlinkStateInternals.java    |   35 +-
 .../streaming/state/StateCheckpointReader.java  |    2 +
 .../streaming/state/StateCheckpointUtils.java   |   16 +-
 .../streaming/state/StateCheckpointWriter.java  |    2 +
 .../apache/beam/runners/flink/AvroITCase.java   |   16 +-
 .../beam/runners/flink/FlattenizeITCase.java    |   14 +-
 .../runners/flink/FlinkRunnerRegistrarTest.java |    9 +-
 .../beam/runners/flink/FlinkTestPipeline.java   |   16 +-
 .../beam/runners/flink/JoinExamplesITCase.java  |   11 +-
 .../runners/flink/MaybeEmptyTestITCase.java     |   13 +-
 .../runners/flink/ParDoMultiOutputITCase.java   |   20 +-
 .../beam/runners/flink/ReadSourceITCase.java    |   22 +-
 .../flink/RemoveDuplicatesEmptyITCase.java      |   14 +-
 .../runners/flink/RemoveDuplicatesITCase.java   |   14 +-
 .../beam/runners/flink/SideInputITCase.java     |   15 +-
 .../apache/beam/runners/flink/TfIdfITCase.java  |   20 +-
 .../beam/runners/flink/WordCountITCase.java     |   16 +-
 .../runners/flink/WordCountJoin2ITCase.java     |   26 +-
 .../runners/flink/WordCountJoin3ITCase.java     |   26 +-
 .../beam/runners/flink/WriteSinkITCase.java     |   20 +-
 .../flink/streaming/GroupAlsoByWindowTest.java  |   37 +-
 .../flink/streaming/GroupByNullKeyTest.java     |   25 +-
 .../flink/streaming/StateSerializationTest.java |   41 +-
 .../streaming/TopWikipediaSessionsITCase.java   |   22 +-
 .../flink/streaming/UnboundedSourceITCase.java  |   44 +-
 .../beam/runners/flink/util/JoinExamples.java   |   33 +-
 runners/google-cloud-dataflow-java/pom.xml      |   12 +-
 .../BlockingDataflowPipelineOptions.java        |   50 -
 .../sdk/options/CloudDebuggerOptions.java       |   53 -
 .../options/DataflowPipelineDebugOptions.java   |  254 --
 .../sdk/options/DataflowPipelineOptions.java    |  115 -
 .../DataflowPipelineWorkerPoolOptions.java      |  258 --
 .../sdk/options/DataflowProfilingOptions.java   |   48 -
 .../options/DataflowWorkerHarnessOptions.java   |   51 -
 .../options/DataflowWorkerLoggingOptions.java   |  155 -
 .../runners/BlockingDataflowPipelineRunner.java |  186 --
 .../DataflowJobAlreadyExistsException.java      |   35 -
 .../DataflowJobAlreadyUpdatedException.java     |   34 -
 .../runners/DataflowJobCancelledException.java  |   39 -
 .../sdk/runners/DataflowJobException.java       |   41 -
 .../runners/DataflowJobExecutionException.java  |   35 -
 .../runners/DataflowJobUpdatedException.java    |   52 -
 .../dataflow/sdk/runners/DataflowPipeline.java  |   60 -
 .../sdk/runners/DataflowPipelineJob.java        |  394 ---
 .../sdk/runners/DataflowPipelineRegistrar.java  |   59 -
 .../sdk/runners/DataflowPipelineRunner.java     | 3009 -----------------
 .../runners/DataflowPipelineRunnerHooks.java    |   39 -
 .../sdk/runners/DataflowPipelineTranslator.java | 1100 -------
 .../sdk/runners/DataflowServiceException.java   |   33 -
 .../sdk/runners/dataflow/AssignWindows.java     |   90 -
 .../runners/dataflow/BigQueryIOTranslator.java  |  126 -
 .../sdk/runners/dataflow/CustomSources.java     |  119 -
 .../dataflow/DataflowAggregatorTransforms.java  |   80 -
 .../dataflow/DataflowMetricUpdateExtractor.java |  111 -
 .../runners/dataflow/PubsubIOTranslator.java    |  108 -
 .../sdk/runners/dataflow/ReadTranslator.java    |  104 -
 .../sdk/runners/dataflow/package-info.java      |   21 -
 .../testing/TestDataflowPipelineOptions.java    |   27 -
 .../sdk/testing/TestDataflowPipelineRunner.java |  256 --
 .../sdk/util/DataflowPathValidator.java         |   98 -
 .../dataflow/sdk/util/DataflowTransport.java    |  112 -
 .../cloud/dataflow/sdk/util/GcsStager.java      |   54 -
 .../cloud/dataflow/sdk/util/MonitoringUtil.java |  235 --
 .../cloud/dataflow/sdk/util/PackageUtil.java    |  328 --
 .../google/cloud/dataflow/sdk/util/Stager.java  |   30 -
 .../BlockingDataflowPipelineOptions.java        |   50 +
 .../beam/sdk/options/CloudDebuggerOptions.java  |   53 +
 .../options/DataflowPipelineDebugOptions.java   |  254 ++
 .../sdk/options/DataflowPipelineOptions.java    |  116 +
 .../DataflowPipelineWorkerPoolOptions.java      |  258 ++
 .../sdk/options/DataflowProfilingOptions.java   |   48 +
 .../options/DataflowWorkerHarnessOptions.java   |   51 +
 .../options/DataflowWorkerLoggingOptions.java   |  155 +
 .../runners/BlockingDataflowPipelineRunner.java |  185 ++
 .../DataflowJobAlreadyExistsException.java      |   35 +
 .../DataflowJobAlreadyUpdatedException.java     |   34 +
 .../runners/DataflowJobCancelledException.java  |   39 +
 .../beam/sdk/runners/DataflowJobException.java  |   41 +
 .../runners/DataflowJobExecutionException.java  |   35 +
 .../runners/DataflowJobUpdatedException.java    |   51 +
 .../beam/sdk/runners/DataflowPipeline.java      |   60 +
 .../beam/sdk/runners/DataflowPipelineJob.java   |  395 +++
 .../sdk/runners/DataflowPipelineRegistrar.java  |   60 +
 .../sdk/runners/DataflowPipelineRunner.java     | 3010 ++++++++++++++++++
 .../runners/DataflowPipelineRunnerHooks.java    |   39 +
 .../sdk/runners/DataflowPipelineTranslator.java | 1102 +++++++
 .../sdk/runners/DataflowServiceException.java   |   33 +
 .../sdk/runners/dataflow/AssignWindows.java     |   89 +
 .../runners/dataflow/BigQueryIOTranslator.java  |  127 +
 .../sdk/runners/dataflow/CustomSources.java     |  121 +
 .../dataflow/DataflowAggregatorTransforms.java  |   81 +
 .../dataflow/DataflowMetricUpdateExtractor.java |  111 +
 .../runners/dataflow/PubsubIOTranslator.java    |  108 +
 .../sdk/runners/dataflow/ReadTranslator.java    |  105 +
 .../beam/sdk/runners/dataflow/package-info.java |   21 +
 .../testing/TestDataflowPipelineOptions.java    |   27 +
 .../sdk/testing/TestDataflowPipelineRunner.java |  257 ++
 .../beam/sdk/util/DataflowPathValidator.java    |   99 +
 .../apache/beam/sdk/util/DataflowTransport.java |  113 +
 .../org/apache/beam/sdk/util/GcsStager.java     |   55 +
 .../apache/beam/sdk/util/MonitoringUtil.java    |  236 ++
 .../org/apache/beam/sdk/util/PackageUtil.java   |  328 ++
 .../java/org/apache/beam/sdk/util/Stager.java   |   30 +
 .../dataflow/sdk/io/DataflowTextIOTest.java     |  118 -
 .../DataflowPipelineDebugOptionsTest.java       |   41 -
 .../options/DataflowPipelineOptionsTest.java    |   92 -
 .../options/DataflowProfilingOptionsTest.java   |   49 -
 .../DataflowWorkerLoggingOptionsTest.java       |   75 -
 .../BlockingDataflowPipelineRunnerTest.java     |  302 --
 .../sdk/runners/DataflowPipelineJobTest.java    |  605 ----
 .../runners/DataflowPipelineRegistrarTest.java  |   73 -
 .../sdk/runners/DataflowPipelineRunnerTest.java | 1369 --------
 .../sdk/runners/DataflowPipelineTest.java       |   45 -
 .../runners/DataflowPipelineTranslatorTest.java |  890 ------
 .../sdk/runners/dataflow/CustomSourcesTest.java |  274 --
 .../testing/TestDataflowPipelineRunnerTest.java |  377 ---
 .../sdk/transforms/DataflowGroupByKeyTest.java  |  111 -
 .../sdk/transforms/DataflowViewTest.java        |  206 --
 .../sdk/util/DataflowPathValidatorTest.java     |   93 -
 .../dataflow/sdk/util/MonitoringUtilTest.java   |  148 -
 .../dataflow/sdk/util/PackageUtilTest.java      |  483 ---
 .../apache/beam/sdk/io/DataflowTextIOTest.java  |  118 +
 .../DataflowPipelineDebugOptionsTest.java       |   41 +
 .../options/DataflowPipelineOptionsTest.java    |   92 +
 .../options/DataflowProfilingOptionsTest.java   |   49 +
 .../DataflowWorkerLoggingOptionsTest.java       |   77 +
 .../BlockingDataflowPipelineRunnerTest.java     |  302 ++
 .../sdk/runners/DataflowPipelineJobTest.java    |  606 ++++
 .../runners/DataflowPipelineRegistrarTest.java  |   74 +
 .../sdk/runners/DataflowPipelineRunnerTest.java | 1371 ++++++++
 .../beam/sdk/runners/DataflowPipelineTest.java  |   45 +
 .../runners/DataflowPipelineTranslatorTest.java |  897 ++++++
 .../sdk/runners/dataflow/CustomSourcesTest.java |  276 ++
 .../testing/TestDataflowPipelineRunnerTest.java |  378 +++
 .../sdk/transforms/DataflowGroupByKeyTest.java  |  110 +
 .../beam/sdk/transforms/DataflowViewTest.java   |  205 ++
 .../sdk/util/DataflowPathValidatorTest.java     |   92 +
 .../beam/sdk/util/MonitoringUtilTest.java       |  149 +
 .../apache/beam/sdk/util/PackageUtilTest.java   |  484 +++
 runners/pom.xml                                 |    2 +-
 .../beam/runners/spark/EvaluationResult.java    |    6 +-
 .../runners/spark/SparkPipelineOptions.java     |   10 +-
 .../beam/runners/spark/SparkPipelineRunner.java |   34 +-
 .../spark/SparkStreamingPipelineOptions.java    |    4 +-
 .../spark/aggregators/NamedAggregators.java     |    7 +-
 .../beam/runners/spark/coders/CoderHelpers.java |    2 +-
 .../runners/spark/coders/NullWritableCoder.java |    2 +-
 .../runners/spark/coders/WritableCoder.java     |    8 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |    6 +-
 .../beam/runners/spark/io/CreateStream.java     |    8 +-
 .../apache/beam/runners/spark/io/KafkaIO.java   |   11 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   14 +-
 .../spark/io/hadoop/ShardNameBuilder.java       |    4 +-
 .../spark/io/hadoop/ShardNameTemplateAware.java |    2 +-
 .../io/hadoop/ShardNameTemplateHelper.java      |    4 +-
 .../runners/spark/translation/DoFnFunction.java |    6 +-
 .../spark/translation/EvaluationContext.java    |   26 +-
 .../spark/translation/MultiDoFnFunction.java    |    6 +-
 .../translation/SparkPipelineEvaluator.java     |   10 +-
 .../SparkPipelineOptionsFactory.java            |    2 +-
 .../SparkPipelineOptionsRegistrar.java          |    4 +-
 .../SparkPipelineRunnerRegistrar.java           |    4 +-
 .../translation/SparkPipelineTranslator.java    |    2 +-
 .../spark/translation/SparkProcessContext.java  |   28 +-
 .../spark/translation/SparkRuntimeContext.java  |   26 +-
 .../spark/translation/TransformEvaluator.java   |    2 +-
 .../spark/translation/TransformTranslator.java  |   62 +-
 .../spark/translation/WindowingHelpers.java     |    2 +-
 .../SparkStreamingPipelineOptionsFactory.java   |    2 +-
 .../SparkStreamingPipelineOptionsRegistrar.java |    4 +-
 .../streaming/StreamingEvaluationContext.java   |   18 +-
 .../streaming/StreamingTransformTranslator.java |   42 +-
 .../StreamingWindowPipelineDetector.java        |   20 +-
 .../runners/spark/util/BroadcastHelper.java     |    2 +-
 .../apache/beam/runners/spark/DeDupTest.java    |   20 +-
 .../beam/runners/spark/EmptyInputTest.java      |   20 +-
 .../beam/runners/spark/SimpleWordCountTest.java |   26 +-
 .../apache/beam/runners/spark/TfIdfTest.java    |   24 +-
 .../runners/spark/coders/WritableCoderTest.java |    2 +-
 .../beam/runners/spark/io/AvroPipelineTest.java |   17 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   36 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   20 +-
 .../spark/io/hadoop/ShardNameBuilderTest.java   |    4 +-
 .../spark/translation/CombineGloballyTest.java  |   20 +-
 .../spark/translation/CombinePerKeyTest.java    |   28 +-
 .../spark/translation/DoFnOutputTest.java       |   16 +-
 .../translation/MultiOutputWordCountTest.java   |   34 +-
 .../spark/translation/SerializationTest.java    |   38 +-
 .../spark/translation/SideEffectsTest.java      |   22 +-
 .../translation/TransformTranslatorTest.java    |   26 +-
 .../translation/WindowedWordCountTest.java      |   21 +-
 .../streaming/FlattenStreamingTest.java         |   30 +-
 .../streaming/KafkaStreamingTest.java           |   43 +-
 .../streaming/SimpleStreamingWordCountTest.java |   30 +-
 .../streaming/utils/EmbeddedKafkaCluster.java   |    7 +-
 .../streaming/utils/PAssertStreaming.java       |    3 +-
 sdks/java/core/pom.xml                          |   16 +-
 .../com/google/cloud/dataflow/sdk/Pipeline.java |  503 ---
 .../cloud/dataflow/sdk/PipelineResult.java      |   96 -
 .../cloud/dataflow/sdk/coders/AtomicCoder.java  |   52 -
 .../cloud/dataflow/sdk/coders/AvroCoder.java    |  715 -----
 .../sdk/coders/BigEndianIntegerCoder.java       |  100 -
 .../dataflow/sdk/coders/BigEndianLongCoder.java |  100 -
 .../dataflow/sdk/coders/ByteArrayCoder.java     |  139 -
 .../cloud/dataflow/sdk/coders/ByteCoder.java    |  112 -
 .../dataflow/sdk/coders/ByteStringCoder.java    |  107 -
 .../sdk/coders/CannotProvideCoderException.java |   96 -
 .../google/cloud/dataflow/sdk/coders/Coder.java |  299 --
 .../dataflow/sdk/coders/CoderException.java     |   37 -
 .../dataflow/sdk/coders/CoderFactories.java     |  275 --
 .../cloud/dataflow/sdk/coders/CoderFactory.java |   44 -
 .../dataflow/sdk/coders/CoderProvider.java      |   34 -
 .../dataflow/sdk/coders/CoderProviders.java     |  165 -
 .../dataflow/sdk/coders/CoderRegistry.java      |  844 -----
 .../dataflow/sdk/coders/CollectionCoder.java    |   74 -
 .../cloud/dataflow/sdk/coders/CustomCoder.java  |  138 -
 .../cloud/dataflow/sdk/coders/DefaultCoder.java |   67 -
 .../dataflow/sdk/coders/DelegateCoder.java      |  165 -
 .../sdk/coders/DeterministicStandardCoder.java  |   39 -
 .../cloud/dataflow/sdk/coders/DoubleCoder.java  |  114 -
 .../dataflow/sdk/coders/DurationCoder.java      |   98 -
 .../cloud/dataflow/sdk/coders/EntityCoder.java  |   87 -
 .../cloud/dataflow/sdk/coders/InstantCoder.java |  114 -
 .../dataflow/sdk/coders/IterableCoder.java      |   79 -
 .../dataflow/sdk/coders/IterableLikeCoder.java  |  279 --
 .../cloud/dataflow/sdk/coders/JAXBCoder.java    |  170 -
 .../cloud/dataflow/sdk/coders/KvCoder.java      |  163 -
 .../cloud/dataflow/sdk/coders/KvCoderBase.java  |   62 -
 .../cloud/dataflow/sdk/coders/ListCoder.java    |   78 -
 .../cloud/dataflow/sdk/coders/MapCoder.java     |  161 -
 .../cloud/dataflow/sdk/coders/MapCoderBase.java |   55 -
 .../dataflow/sdk/coders/NullableCoder.java      |  179 --
 .../cloud/dataflow/sdk/coders/Proto2Coder.java  |  363 ---
 .../dataflow/sdk/coders/SerializableCoder.java  |  184 --
 .../cloud/dataflow/sdk/coders/SetCoder.java     |   95 -
 .../dataflow/sdk/coders/StandardCoder.java      |  230 --
 .../sdk/coders/StringDelegateCoder.java         |   87 -
 .../dataflow/sdk/coders/StringUtf8Coder.java    |  140 -
 .../sdk/coders/StructuralByteArray.java         |   58 -
 .../dataflow/sdk/coders/TableRowJsonCoder.java  |   83 -
 .../sdk/coders/TextualIntegerCoder.java         |   70 -
 .../cloud/dataflow/sdk/coders/VarIntCoder.java  |   98 -
 .../cloud/dataflow/sdk/coders/VarLongCoder.java |   97 -
 .../cloud/dataflow/sdk/coders/VoidCoder.java    |   77 -
 .../cloud/dataflow/sdk/coders/package-info.java |   45 -
 .../sdk/coders/protobuf/ProtoCoder.java         |  406 ---
 .../sdk/coders/protobuf/ProtobufUtil.java       |  172 -
 .../sdk/coders/protobuf/package-info.java       |   24 -
 .../google/cloud/dataflow/sdk/io/AvroIO.java    |  811 -----
 .../cloud/dataflow/sdk/io/AvroSource.java       |  648 ----
 .../cloud/dataflow/sdk/io/BigQueryIO.java       | 1685 ----------
 .../cloud/dataflow/sdk/io/BlockBasedSource.java |  238 --
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  272 --
 .../cloud/dataflow/sdk/io/BoundedSource.java    |  278 --
 .../cloud/dataflow/sdk/io/CompressedSource.java |  414 ---
 .../cloud/dataflow/sdk/io/CountingInput.java    |  224 --
 .../cloud/dataflow/sdk/io/CountingSource.java   |  481 ---
 .../cloud/dataflow/sdk/io/DatastoreIO.java      |  947 ------
 .../cloud/dataflow/sdk/io/FileBasedSink.java    |  865 -----
 .../cloud/dataflow/sdk/io/FileBasedSource.java  |  663 ----
 .../dataflow/sdk/io/OffsetBasedSource.java      |  329 --
 .../cloud/dataflow/sdk/io/PubsubClient.java     |  322 --
 .../cloud/dataflow/sdk/io/PubsubGrpcClient.java |  401 ---
 .../google/cloud/dataflow/sdk/io/PubsubIO.java  | 1048 ------
 .../com/google/cloud/dataflow/sdk/io/Read.java  |  254 --
 .../dataflow/sdk/io/ShardNameTemplate.java      |   76 -
 .../com/google/cloud/dataflow/sdk/io/Sink.java  |  255 --
 .../google/cloud/dataflow/sdk/io/Source.java    |  194 --
 .../google/cloud/dataflow/sdk/io/TextIO.java    |  991 ------
 .../cloud/dataflow/sdk/io/UnboundedSource.java  |  254 --
 .../com/google/cloud/dataflow/sdk/io/Write.java |  217 --
 .../google/cloud/dataflow/sdk/io/XmlSink.java   |  311 --
 .../google/cloud/dataflow/sdk/io/XmlSource.java |  544 ----
 .../dataflow/sdk/io/bigtable/BigtableIO.java    |  989 ------
 .../sdk/io/bigtable/BigtableService.java        |  110 -
 .../sdk/io/bigtable/BigtableServiceImpl.java    |  243 --
 .../dataflow/sdk/io/bigtable/package-info.java  |   23 -
 .../cloud/dataflow/sdk/io/package-info.java     |   38 -
 .../cloud/dataflow/sdk/io/range/ByteKey.java    |  174 -
 .../dataflow/sdk/io/range/ByteKeyRange.java     |  377 ---
 .../sdk/io/range/ByteKeyRangeTracker.java       |  118 -
 .../sdk/io/range/OffsetRangeTracker.java        |  183 --
 .../dataflow/sdk/io/range/RangeTracker.java     |  221 --
 .../dataflow/sdk/io/range/package-info.java     |   24 -
 .../sdk/options/ApplicationNameOptions.java     |   34 -
 .../dataflow/sdk/options/BigQueryOptions.java   |   32 -
 .../cloud/dataflow/sdk/options/Default.java     |  154 -
 .../sdk/options/DefaultValueFactory.java        |   39 -
 .../cloud/dataflow/sdk/options/Description.java |   36 -
 .../sdk/options/DirectPipelineOptions.java      |   76 -
 .../cloud/dataflow/sdk/options/GcpOptions.java  |  292 --
 .../cloud/dataflow/sdk/options/GcsOptions.java  |  114 -
 .../sdk/options/GoogleApiDebugOptions.java      |   88 -
 .../cloud/dataflow/sdk/options/Hidden.java      |   34 -
 .../dataflow/sdk/options/PipelineOptions.java   |  264 --
 .../sdk/options/PipelineOptionsFactory.java     | 1490 ---------
 .../sdk/options/PipelineOptionsRegistrar.java   |   37 -
 .../sdk/options/PipelineOptionsValidator.java   |  103 -
 .../sdk/options/ProxyInvocationHandler.java     |  442 ---
 .../dataflow/sdk/options/PubsubOptions.java     |   36 -
 .../dataflow/sdk/options/StreamingOptions.java  |   31 -
 .../cloud/dataflow/sdk/options/Validation.java  |   47 -
 .../dataflow/sdk/options/package-info.java      |   26 -
 .../google/cloud/dataflow/sdk/package-info.java |   34 -
 .../runners/AggregatorPipelineExtractor.java    |   98 -
 .../runners/AggregatorRetrievalException.java   |   33 -
 .../dataflow/sdk/runners/AggregatorValues.java  |   53 -
 .../dataflow/sdk/runners/DirectPipeline.java    |   56 -
 .../sdk/runners/DirectPipelineRegistrar.java    |   54 -
 .../sdk/runners/DirectPipelineRunner.java       | 1320 --------
 .../dataflow/sdk/runners/PipelineRunner.java    |   77 -
 .../sdk/runners/PipelineRunnerRegistrar.java    |   41 -
 .../sdk/runners/RecordingPipelineVisitor.java   |   55 -
 .../sdk/runners/TransformHierarchy.java         |  105 -
 .../dataflow/sdk/runners/TransformTreeNode.java |  253 --
 .../inprocess/AbstractModelEnforcement.java     |   38 -
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 -
 .../inprocess/BoundedReadEvaluatorFactory.java  |  156 -
 .../sdk/runners/inprocess/BundleFactory.java    |   50 -
 .../CachedThreadPoolExecutorServiceFactory.java |   44 -
 .../dataflow/sdk/runners/inprocess/Clock.java   |   31 -
 .../runners/inprocess/CompletionCallback.java   |   36 -
 .../ConsumerTrackingPipelineVisitor.java        |  175 -
 .../inprocess/EmptyTransformEvaluator.java      |   51 -
 .../EncodabilityEnforcementFactory.java         |   71 -
 .../sdk/runners/inprocess/EvaluatorKey.java     |   57 -
 .../inprocess/ExecutorServiceFactory.java       |   34 -
 .../ExecutorServiceParallelExecutor.java        |  478 ---
 .../inprocess/FlattenEvaluatorFactory.java      |   85 -
 .../runners/inprocess/ForwardingPTransform.java |   56 -
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  273 --
 .../ImmutabilityCheckingBundleFactory.java      |  131 -
 .../ImmutabilityEnforcementFactory.java         |  104 -
 .../inprocess/InMemoryWatermarkManager.java     | 1313 --------
 .../sdk/runners/inprocess/InProcessBundle.java  |  123 -
 .../inprocess/InProcessBundleFactory.java       |  157 -
 .../inprocess/InProcessBundleOutputManager.java |   52 -
 .../sdk/runners/inprocess/InProcessCreate.java  |  235 --
 .../inprocess/InProcessEvaluationContext.java   |  411 ---
 .../inprocess/InProcessExecutionContext.java    |  108 -
 .../runners/inprocess/InProcessExecutor.java    |   48 -
 .../inprocess/InProcessPipelineOptions.java     |  101 -
 .../inprocess/InProcessPipelineRunner.java      |  366 ---
 .../runners/inprocess/InProcessRegistrar.java   |   54 -
 .../inprocess/InProcessSideInputContainer.java  |  232 --
 .../inprocess/InProcessTimerInternals.java      |   85 -
 .../inprocess/InProcessTransformResult.java     |   77 -
 .../inprocess/KeyedPValueTrackingVisitor.java   |   97 -
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 -
 .../inprocess/ModelEnforcementFactory.java      |   30 -
 .../sdk/runners/inprocess/NanosOffsetClock.java |   60 -
 .../inprocess/PTransformOverrideFactory.java    |   33 -
 .../inprocess/ParDoInProcessEvaluator.java      |  120 -
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   92 -
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   89 -
 .../PassthroughTransformEvaluator.java          |   49 -
 .../runners/inprocess/ShardControlledWrite.java |   81 -
 .../sdk/runners/inprocess/StepAndKey.java       |   70 -
 .../runners/inprocess/StepTransformResult.java  |  164 -
 .../inprocess/TextIOShardedWriteFactory.java    |   78 -
 .../runners/inprocess/TransformEvaluator.java   |   47 -
 .../inprocess/TransformEvaluatorFactory.java    |   44 -
 .../inprocess/TransformEvaluatorRegistry.java   |   76 -
 .../runners/inprocess/TransformExecutor.java    |  175 -
 .../inprocess/TransformExecutorService.java     |   36 -
 .../inprocess/TransformExecutorServices.java    |  155 -
 .../UnboundedReadEvaluatorFactory.java          |  178 --
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 -
 .../inprocess/WatermarkCallbackExecutor.java    |  145 -
 .../inprocess/WindowEvaluatorFactory.java       |  131 -
 .../dataflow/sdk/runners/package-info.java      |   34 -
 .../dataflow/sdk/runners/worker/IsmFormat.java  |  948 ------
 .../sdk/runners/worker/package-info.java        |   25 -
 .../dataflow/sdk/testing/CoderProperties.java   |  350 --
 .../cloud/dataflow/sdk/testing/PAssert.java     |  825 -----
 .../dataflow/sdk/testing/RunnableOnService.java |   34 -
 .../sdk/testing/SerializableMatcher.java        |   37 -
 .../sdk/testing/SerializableMatchers.java       | 1181 -------
 .../dataflow/sdk/testing/SourceTestUtils.java   |  676 ----
 .../dataflow/sdk/testing/TestPipeline.java      |  205 --
 .../dataflow/sdk/testing/WindowFnTestUtils.java |  326 --
 .../dataflow/sdk/testing/package-info.java      |   22 -
 .../dataflow/sdk/transforms/Aggregator.java     |   79 -
 .../sdk/transforms/AggregatorRetriever.java     |   37 -
 .../sdk/transforms/AppliedPTransform.java       |  101 -
 .../sdk/transforms/ApproximateQuantiles.java    |  767 -----
 .../sdk/transforms/ApproximateUnique.java       |  420 ---
 .../cloud/dataflow/sdk/transforms/Combine.java  | 2242 -------------
 .../dataflow/sdk/transforms/CombineFnBase.java  |  285 --
 .../dataflow/sdk/transforms/CombineFns.java     | 1010 ------
 .../sdk/transforms/CombineWithContext.java      |  279 --
 .../cloud/dataflow/sdk/transforms/Count.java    |  193 --
 .../cloud/dataflow/sdk/transforms/Create.java   |  427 ---
 .../cloud/dataflow/sdk/transforms/DoFn.java     |  564 ----
 .../dataflow/sdk/transforms/DoFnReflector.java  |  669 ----
 .../dataflow/sdk/transforms/DoFnTester.java     |  496 ---
 .../sdk/transforms/DoFnWithContext.java         |  417 ---
 .../cloud/dataflow/sdk/transforms/Filter.java   |  235 --
 .../sdk/transforms/FlatMapElements.java         |  146 -
 .../cloud/dataflow/sdk/transforms/Flatten.java  |  220 --
 .../dataflow/sdk/transforms/GroupByKey.java     |  276 --
 .../transforms/IntraBundleParallelization.java  |  348 --
 .../cloud/dataflow/sdk/transforms/Keys.java     |   69 -
 .../cloud/dataflow/sdk/transforms/KvSwap.java   |   74 -
 .../dataflow/sdk/transforms/MapElements.java    |  113 -
 .../cloud/dataflow/sdk/transforms/Max.java      |  256 --
 .../cloud/dataflow/sdk/transforms/Mean.java     |  203 --
 .../cloud/dataflow/sdk/transforms/Min.java      |  256 --
 .../dataflow/sdk/transforms/PTransform.java     |  325 --
 .../cloud/dataflow/sdk/transforms/ParDo.java    | 1323 --------
 .../dataflow/sdk/transforms/Partition.java      |  174 -
 .../sdk/transforms/RemoveDuplicates.java        |  159 -
 .../cloud/dataflow/sdk/transforms/Sample.java   |  247 --
 .../sdk/transforms/SerializableComparator.java  |   29 -
 .../sdk/transforms/SerializableFunction.java    |   32 -
 .../dataflow/sdk/transforms/SimpleFunction.java |   55 -
 .../cloud/dataflow/sdk/transforms/Sum.java      |  189 --
 .../cloud/dataflow/sdk/transforms/Top.java      |  560 ----
 .../cloud/dataflow/sdk/transforms/Values.java   |   69 -
 .../cloud/dataflow/sdk/transforms/View.java     |  471 ---
 .../cloud/dataflow/sdk/transforms/WithKeys.java |  141 -
 .../dataflow/sdk/transforms/WithTimestamps.java |  130 -
 .../cloud/dataflow/sdk/transforms/Write.java    |   30 -
 .../sdk/transforms/display/DisplayData.java     |  901 ------
 .../sdk/transforms/display/HasDisplayData.java  |   55 -
 .../sdk/transforms/join/CoGbkResult.java        |  464 ---
 .../sdk/transforms/join/CoGbkResultSchema.java  |  135 -
 .../sdk/transforms/join/CoGroupByKey.java       |  212 --
 .../transforms/join/KeyedPCollectionTuple.java  |  248 --
 .../sdk/transforms/join/RawUnionValue.java      |   52 -
 .../sdk/transforms/join/UnionCoder.java         |  148 -
 .../sdk/transforms/join/package-info.java       |   22 -
 .../dataflow/sdk/transforms/package-info.java   |   44 -
 .../sdk/transforms/windowing/AfterAll.java      |  114 -
 .../windowing/AfterDelayFromFirstElement.java   |  323 --
 .../sdk/transforms/windowing/AfterEach.java     |  133 -
 .../sdk/transforms/windowing/AfterFirst.java    |  117 -
 .../sdk/transforms/windowing/AfterPane.java     |  143 -
 .../windowing/AfterProcessingTime.java          |   96 -
 .../AfterSynchronizedProcessingTime.java        |   76 -
 .../transforms/windowing/AfterWatermark.java    |  396 ---
 .../sdk/transforms/windowing/BoundedWindow.java |   47 -
 .../transforms/windowing/CalendarWindows.java   |  349 --
 .../transforms/windowing/DefaultTrigger.java    |   94 -
 .../sdk/transforms/windowing/FixedWindows.java  |  117 -
 .../sdk/transforms/windowing/GlobalWindow.java  |   69 -
 .../sdk/transforms/windowing/GlobalWindows.java |   64 -
 .../transforms/windowing/IntervalWindow.java    |  202 --
 .../transforms/windowing/InvalidWindows.java    |   88 -
 .../MergeOverlappingIntervalWindows.java        |   87 -
 .../windowing/NonMergingWindowFn.java           |   36 -
 .../transforms/windowing/OrFinallyTrigger.java  |  102 -
 .../sdk/transforms/windowing/OutputTimeFn.java  |  320 --
 .../sdk/transforms/windowing/OutputTimeFns.java |  169 -
 .../sdk/transforms/windowing/PaneInfo.java      |  385 ---
 .../windowing/PartitioningWindowFn.java         |   62 -
 .../sdk/transforms/windowing/Repeatedly.java    |   98 -
 .../sdk/transforms/windowing/Sessions.java      |  113 -
 .../transforms/windowing/SlidingWindows.java    |  215 --
 .../sdk/transforms/windowing/Trigger.java       |  535 ----
 .../transforms/windowing/TriggerBuilder.java    |   29 -
 .../sdk/transforms/windowing/Window.java        |  654 ----
 .../sdk/transforms/windowing/WindowFn.java      |  222 --
 .../sdk/transforms/windowing/package-info.java  |   50 -
 .../dataflow/sdk/util/ActiveWindowSet.java      |  182 --
 .../cloud/dataflow/sdk/util/ApiSurface.java     |  643 ----
 .../dataflow/sdk/util/AppEngineEnvironment.java |   62 -
 .../dataflow/sdk/util/AppliedCombineFn.java     |  132 -
 .../cloud/dataflow/sdk/util/AssignWindows.java  |   46 -
 .../dataflow/sdk/util/AssignWindowsDoFn.java    |   68 -
 ...AttemptAndTimeBoundedExponentialBackOff.java |  169 -
 .../util/AttemptBoundedExponentialBackOff.java  |   84 -
 .../cloud/dataflow/sdk/util/AvroUtils.java      |  346 --
 .../dataflow/sdk/util/BaseExecutionContext.java |  156 -
 .../dataflow/sdk/util/BatchTimerInternals.java  |  139 -
 .../dataflow/sdk/util/BigQueryServices.java     |   61 -
 .../dataflow/sdk/util/BigQueryServicesImpl.java |  196 --
 .../sdk/util/BigQueryTableInserter.java         |  459 ---
 .../sdk/util/BigQueryTableRowIterator.java      |  472 ---
 .../cloud/dataflow/sdk/util/BitSetCoder.java    |   61 -
 .../BufferedElementCountingOutputStream.java    |  187 --
 .../cloud/dataflow/sdk/util/CloudKnownType.java |  139 -
 .../cloud/dataflow/sdk/util/CloudObject.java    |  185 --
 .../cloud/dataflow/sdk/util/CoderUtils.java     |  328 --
 .../sdk/util/CombineContextFactory.java         |  109 -
 .../cloud/dataflow/sdk/util/CombineFnUtil.java  |  205 --
 .../dataflow/sdk/util/CounterAggregator.java    |   97 -
 .../dataflow/sdk/util/CredentialFactory.java    |   30 -
 .../cloud/dataflow/sdk/util/Credentials.java    |  193 --
 .../dataflow/sdk/util/DataflowReleaseInfo.java  |   88 -
 .../sdk/util/DirectModeExecutionContext.java    |  131 -
 .../sdk/util/DirectSideInputReader.java         |   74 -
 .../cloud/dataflow/sdk/util/DoFnInfo.java       |   68 -
 .../cloud/dataflow/sdk/util/DoFnRunner.java     |   62 -
 .../cloud/dataflow/sdk/util/DoFnRunnerBase.java |  559 ----
 .../cloud/dataflow/sdk/util/DoFnRunners.java    |  144 -
 .../dataflow/sdk/util/ExecutableTrigger.java    |  158 -
 .../dataflow/sdk/util/ExecutionContext.java     |  103 -
 .../sdk/util/ExposedByteArrayInputStream.java   |   52 -
 .../sdk/util/ExposedByteArrayOutputStream.java  |  118 -
 .../dataflow/sdk/util/FileIOChannelFactory.java |  136 -
 .../dataflow/sdk/util/FinishedTriggers.java     |   44 -
 .../sdk/util/FinishedTriggersBitSet.java        |   69 -
 .../dataflow/sdk/util/FinishedTriggersSet.java  |   75 -
 .../dataflow/sdk/util/GcpCredentialFactory.java |   46 -
 .../dataflow/sdk/util/GcsIOChannelFactory.java  |   87 -
 .../google/cloud/dataflow/sdk/util/GcsUtil.java |  411 ---
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |  106 -
 .../sdk/util/GroupAlsoByWindowsDoFn.java        |   59 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   99 -
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  246 --
 .../dataflow/sdk/util/IOChannelFactory.java     |  102 -
 .../cloud/dataflow/sdk/util/IOChannelUtils.java |  205 --
 .../sdk/util/IllegalMutationException.java      |   53 -
 .../dataflow/sdk/util/InstanceBuilder.java      |  270 --
 .../util/IntervalBoundedExponentialBackOff.java |   94 -
 .../cloud/dataflow/sdk/util/KeyedWorkItem.java  |   43 -
 .../dataflow/sdk/util/KeyedWorkItemCoder.java   |  122 -
 .../cloud/dataflow/sdk/util/KeyedWorkItems.java |  122 -
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  146 -
 .../dataflow/sdk/util/MapAggregatorValues.java  |   49 -
 .../sdk/util/MergingActiveWindowSet.java        |  575 ----
 .../cloud/dataflow/sdk/util/MimeTypes.java      |   24 -
 .../dataflow/sdk/util/MutationDetector.java     |   32 -
 .../dataflow/sdk/util/MutationDetectors.java    |  182 --
 .../cloud/dataflow/sdk/util/NonEmptyPanes.java  |  150 -
 .../sdk/util/NonMergingActiveWindowSet.java     |   92 -
 .../sdk/util/NoopCredentialFactory.java         |   39 -
 .../dataflow/sdk/util/NoopPathValidator.java    |   49 -
 .../dataflow/sdk/util/NullSideInputReader.java  |   62 -
 .../dataflow/sdk/util/OutputReference.java      |   43 -
 .../sdk/util/PCollectionViewWindow.java         |   68 -
 .../dataflow/sdk/util/PCollectionViews.java     |  427 ---
 .../google/cloud/dataflow/sdk/util/PTuple.java  |  161 -
 .../dataflow/sdk/util/PaneInfoTracker.java      |  153 -
 .../cloud/dataflow/sdk/util/PathValidator.java  |   48 -
 .../sdk/util/PerKeyCombineFnRunner.java         |  149 -
 .../sdk/util/PerKeyCombineFnRunners.java        |  259 --
 .../cloud/dataflow/sdk/util/PropertyNames.java  |  108 -
 .../dataflow/sdk/util/RandomAccessData.java     |  354 --
 .../cloud/dataflow/sdk/util/ReduceFn.java       |  130 -
 .../sdk/util/ReduceFnContextFactory.java        |  496 ---
 .../cloud/dataflow/sdk/util/ReduceFnRunner.java |  903 ------
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |   47 -
 .../cloud/dataflow/sdk/util/Reshuffle.java      |  147 -
 .../dataflow/sdk/util/ReshuffleTrigger.java     |   63 -
 .../sdk/util/RetryHttpRequestInitializer.java   |  251 --
 .../dataflow/sdk/util/SerializableUtils.java    |  160 -
 .../cloud/dataflow/sdk/util/Serializer.java     |  146 -
 .../sdk/util/ShardingWritableByteChannel.java   |  119 -
 .../dataflow/sdk/util/SideInputReader.java      |   49 -
 .../dataflow/sdk/util/SimpleDoFnRunner.java     |   57 -
 .../cloud/dataflow/sdk/util/StreamUtils.java    |   69 -
 .../cloud/dataflow/sdk/util/StringUtils.java    |  243 --
 .../google/cloud/dataflow/sdk/util/Structs.java |  385 ---
 .../dataflow/sdk/util/SystemDoFnInternal.java   |   39 -
 .../cloud/dataflow/sdk/util/SystemReduceFn.java |  135 -
 .../cloud/dataflow/sdk/util/TestCredential.java |   52 -
 .../cloud/dataflow/sdk/util/TimeDomain.java     |   43 -
 .../cloud/dataflow/sdk/util/TimeUtil.java       |  166 -
 .../cloud/dataflow/sdk/util/TimerInternals.java |  270 --
 .../google/cloud/dataflow/sdk/util/Timers.java  |   60 -
 .../cloud/dataflow/sdk/util/Transport.java      |  151 -
 .../sdk/util/TriggerContextFactory.java         |  512 ---
 .../cloud/dataflow/sdk/util/TriggerRunner.java  |  233 --
 .../dataflow/sdk/util/UnownedInputStream.java   |   77 -
 .../dataflow/sdk/util/UnownedOutputStream.java  |   57 -
 .../sdk/util/UploadIdResponseInterceptor.java   |   62 -
 .../dataflow/sdk/util/UserCodeException.java    |   95 -
 .../dataflow/sdk/util/ValueWithRecordId.java    |  155 -
 .../google/cloud/dataflow/sdk/util/Values.java  |   89 -
 .../google/cloud/dataflow/sdk/util/VarInt.java  |  116 -
 .../cloud/dataflow/sdk/util/WatermarkHold.java  |  476 ---
 .../cloud/dataflow/sdk/util/Weighted.java       |   28 -
 .../cloud/dataflow/sdk/util/WeightedValue.java  |   46 -
 .../cloud/dataflow/sdk/util/WindowTracing.java  |   37 -
 .../cloud/dataflow/sdk/util/WindowedValue.java  |  721 -----
 .../dataflow/sdk/util/WindowingInternals.java   |   83 -
 .../dataflow/sdk/util/WindowingStrategy.java    |  267 --
 .../cloud/dataflow/sdk/util/ZipFiles.java       |  295 --
 .../cloud/dataflow/sdk/util/common/Counter.java | 1104 -------
 .../sdk/util/common/CounterProvider.java        |   27 -
 .../dataflow/sdk/util/common/CounterSet.java    |  178 --
 .../util/common/ElementByteSizeObservable.java  |   42 -
 .../ElementByteSizeObservableIterable.java      |   64 -
 .../ElementByteSizeObservableIterator.java      |   37 -
 .../util/common/ElementByteSizeObserver.java    |   93 -
 .../sdk/util/common/PeekingReiterator.java      |   99 -
 .../sdk/util/common/ReflectHelpers.java         |  211 --
 .../dataflow/sdk/util/common/Reiterable.java    |   28 -
 .../dataflow/sdk/util/common/Reiterator.java    |   40 -
 .../dataflow/sdk/util/common/package-info.java  |   19 -
 .../sdk/util/common/worker/StateSampler.java    |  366 ---
 .../sdk/util/common/worker/package-info.java    |   19 -
 .../cloud/dataflow/sdk/util/gcsfs/GcsPath.java  |  620 ----
 .../dataflow/sdk/util/gcsfs/package-info.java   |   19 -
 .../cloud/dataflow/sdk/util/package-info.java   |   19 -
 .../util/state/AccumulatorCombiningState.java   |   53 -
 .../cloud/dataflow/sdk/util/state/BagState.java |   28 -
 .../dataflow/sdk/util/state/CombiningState.java |   42 -
 .../CopyOnAccessInMemoryStateInternals.java     |  456 ---
 .../sdk/util/state/InMemoryStateInternals.java  |  416 ---
 .../sdk/util/state/MergingStateAccessor.java    |   41 -
 .../dataflow/sdk/util/state/ReadableState.java  |   55 -
 .../cloud/dataflow/sdk/util/state/State.java    |   32 -
 .../dataflow/sdk/util/state/StateAccessor.java  |   37 -
 .../dataflow/sdk/util/state/StateContext.java   |   43 -
 .../dataflow/sdk/util/state/StateContexts.java  |  109 -
 .../dataflow/sdk/util/state/StateInternals.java |   57 -
 .../dataflow/sdk/util/state/StateMerging.java   |  256 --
 .../dataflow/sdk/util/state/StateNamespace.java |   56 -
 .../sdk/util/state/StateNamespaceForTest.java   |   65 -
 .../sdk/util/state/StateNamespaces.java         |  279 --
 .../dataflow/sdk/util/state/StateTable.java     |   91 -
 .../cloud/dataflow/sdk/util/state/StateTag.java |   98 -
 .../dataflow/sdk/util/state/StateTags.java      |  581 ----
 .../dataflow/sdk/util/state/ValueState.java     |   37 -
 .../sdk/util/state/WatermarkHoldState.java      |   44 -
 .../google/cloud/dataflow/sdk/values/KV.java    |  131 -
 .../cloud/dataflow/sdk/values/PBegin.java       |   88 -
 .../cloud/dataflow/sdk/values/PCollection.java  |  251 --
 .../dataflow/sdk/values/PCollectionList.java    |  239 --
 .../dataflow/sdk/values/PCollectionTuple.java   |  265 --
 .../dataflow/sdk/values/PCollectionView.java    |   65 -
 .../google/cloud/dataflow/sdk/values/PDone.java |   48 -
 .../cloud/dataflow/sdk/values/PInput.java       |   57 -
 .../cloud/dataflow/sdk/values/POutput.java      |   77 -
 .../dataflow/sdk/values/POutputValueBase.java   |  103 -
 .../cloud/dataflow/sdk/values/PValue.java       |   39 -
 .../cloud/dataflow/sdk/values/PValueBase.java   |  156 -
 .../dataflow/sdk/values/TimestampedValue.java   |  156 -
 .../cloud/dataflow/sdk/values/TupleTag.java     |  197 --
 .../cloud/dataflow/sdk/values/TupleTagList.java |  149 -
 .../dataflow/sdk/values/TypeDescriptor.java     |  352 --
 .../cloud/dataflow/sdk/values/TypedPValue.java  |  198 --
 .../cloud/dataflow/sdk/values/package-info.java |   53 -
 .../main/java/org/apache/beam/sdk/Pipeline.java |  504 +++
 .../org/apache/beam/sdk/PipelineResult.java     |   96 +
 .../org/apache/beam/sdk/coders/AtomicCoder.java |   52 +
 .../org/apache/beam/sdk/coders/AvroCoder.java   |  715 +++++
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |  100 +
 .../beam/sdk/coders/BigEndianLongCoder.java     |  100 +
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |  140 +
 .../org/apache/beam/sdk/coders/ByteCoder.java   |  112 +
 .../apache/beam/sdk/coders/ByteStringCoder.java |  108 +
 .../sdk/coders/CannotProvideCoderException.java |   96 +
 .../java/org/apache/beam/sdk/coders/Coder.java  |  300 ++
 .../apache/beam/sdk/coders/CoderException.java  |   37 +
 .../apache/beam/sdk/coders/CoderFactories.java  |  275 ++
 .../apache/beam/sdk/coders/CoderFactory.java    |   44 +
 .../apache/beam/sdk/coders/CoderProvider.java   |   34 +
 .../apache/beam/sdk/coders/CoderProviders.java  |  166 +
 .../apache/beam/sdk/coders/CoderRegistry.java   |  845 +++++
 .../apache/beam/sdk/coders/CollectionCoder.java |   75 +
 .../org/apache/beam/sdk/coders/CustomCoder.java |  140 +
 .../apache/beam/sdk/coders/DefaultCoder.java    |   67 +
 .../apache/beam/sdk/coders/DelegateCoder.java   |  165 +
 .../sdk/coders/DeterministicStandardCoder.java  |   39 +
 .../org/apache/beam/sdk/coders/DoubleCoder.java |  114 +
 .../apache/beam/sdk/coders/DurationCoder.java   |   98 +
 .../org/apache/beam/sdk/coders/EntityCoder.java |   87 +
 .../apache/beam/sdk/coders/InstantCoder.java    |  115 +
 .../apache/beam/sdk/coders/IterableCoder.java   |   80 +
 .../beam/sdk/coders/IterableLikeCoder.java      |  280 ++
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |  171 +
 .../org/apache/beam/sdk/coders/KvCoder.java     |  164 +
 .../org/apache/beam/sdk/coders/KvCoderBase.java |   62 +
 .../org/apache/beam/sdk/coders/ListCoder.java   |   79 +
 .../org/apache/beam/sdk/coders/MapCoder.java    |  162 +
 .../apache/beam/sdk/coders/MapCoderBase.java    |   55 +
 .../apache/beam/sdk/coders/NullableCoder.java   |  180 ++
 .../org/apache/beam/sdk/coders/Proto2Coder.java |  364 +++
 .../beam/sdk/coders/SerializableCoder.java      |  184 ++
 .../org/apache/beam/sdk/coders/SetCoder.java    |   96 +
 .../apache/beam/sdk/coders/StandardCoder.java   |  232 ++
 .../beam/sdk/coders/StringDelegateCoder.java    |   87 +
 .../apache/beam/sdk/coders/StringUtf8Coder.java |  141 +
 .../beam/sdk/coders/StructuralByteArray.java    |   58 +
 .../beam/sdk/coders/TableRowJsonCoder.java      |   83 +
 .../beam/sdk/coders/TextualIntegerCoder.java    |   70 +
 .../org/apache/beam/sdk/coders/VarIntCoder.java |   98 +
 .../apache/beam/sdk/coders/VarLongCoder.java    |   97 +
 .../org/apache/beam/sdk/coders/VoidCoder.java   |   77 +
 .../apache/beam/sdk/coders/package-info.java    |   45 +
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |  407 +++
 .../beam/sdk/coders/protobuf/ProtobufUtil.java  |  173 +
 .../beam/sdk/coders/protobuf/package-info.java  |   24 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  812 +++++
 .../java/org/apache/beam/sdk/io/AvroSource.java |  649 ++++
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 1686 ++++++++++
 .../apache/beam/sdk/io/BlockBasedSource.java    |  238 ++
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  273 ++
 .../org/apache/beam/sdk/io/BoundedSource.java   |  278 ++
 .../apache/beam/sdk/io/CompressedSource.java    |  415 +++
 .../org/apache/beam/sdk/io/CountingInput.java   |  225 ++
 .../org/apache/beam/sdk/io/CountingSource.java  |  482 +++
 .../org/apache/beam/sdk/io/DatastoreIO.java     |  947 ++++++
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  866 +++++
 .../org/apache/beam/sdk/io/FileBasedSource.java |  664 ++++
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  330 ++
 .../org/apache/beam/sdk/io/PubsubClient.java    |  323 ++
 .../apache/beam/sdk/io/PubsubGrpcClient.java    |  403 +++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1049 ++++++
 .../main/java/org/apache/beam/sdk/io/Read.java  |  254 ++
 .../apache/beam/sdk/io/ShardNameTemplate.java   |   76 +
 .../main/java/org/apache/beam/sdk/io/Sink.java  |  255 ++
 .../java/org/apache/beam/sdk/io/Source.java     |  194 ++
 .../java/org/apache/beam/sdk/io/TextIO.java     |  992 ++++++
 .../org/apache/beam/sdk/io/UnboundedSource.java |  254 ++
 .../main/java/org/apache/beam/sdk/io/Write.java |  216 ++
 .../java/org/apache/beam/sdk/io/XmlSink.java    |  312 ++
 .../java/org/apache/beam/sdk/io/XmlSource.java  |  545 ++++
 .../apache/beam/sdk/io/bigtable/BigtableIO.java |  989 ++++++
 .../beam/sdk/io/bigtable/BigtableService.java   |  111 +
 .../sdk/io/bigtable/BigtableServiceImpl.java    |  244 ++
 .../beam/sdk/io/bigtable/package-info.java      |   23 +
 .../org/apache/beam/sdk/io/package-info.java    |   38 +
 .../org/apache/beam/sdk/io/range/ByteKey.java   |  174 +
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |  377 +++
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |  118 +
 .../beam/sdk/io/range/OffsetRangeTracker.java   |  183 ++
 .../apache/beam/sdk/io/range/RangeTracker.java  |  221 ++
 .../apache/beam/sdk/io/range/package-info.java  |   24 +
 .../sdk/options/ApplicationNameOptions.java     |   34 +
 .../beam/sdk/options/BigQueryOptions.java       |   32 +
 .../org/apache/beam/sdk/options/Default.java    |  154 +
 .../beam/sdk/options/DefaultValueFactory.java   |   39 +
 .../apache/beam/sdk/options/Description.java    |   36 +
 .../beam/sdk/options/DirectPipelineOptions.java |   76 +
 .../org/apache/beam/sdk/options/GcpOptions.java |  293 ++
 .../org/apache/beam/sdk/options/GcsOptions.java |  115 +
 .../beam/sdk/options/GoogleApiDebugOptions.java |   88 +
 .../org/apache/beam/sdk/options/Hidden.java     |   34 +
 .../beam/sdk/options/PipelineOptions.java       |  265 ++
 .../sdk/options/PipelineOptionsFactory.java     | 1491 +++++++++
 .../sdk/options/PipelineOptionsRegistrar.java   |   37 +
 .../sdk/options/PipelineOptionsValidator.java   |  104 +
 .../sdk/options/ProxyInvocationHandler.java     |  443 +++
 .../apache/beam/sdk/options/PubsubOptions.java  |   36 +
 .../beam/sdk/options/StreamingOptions.java      |   31 +
 .../org/apache/beam/sdk/options/Validation.java |   47 +
 .../apache/beam/sdk/options/package-info.java   |   26 +
 .../java/org/apache/beam/sdk/package-info.java  |   34 +
 .../runners/AggregatorPipelineExtractor.java    |   99 +
 .../runners/AggregatorRetrievalException.java   |   33 +
 .../beam/sdk/runners/AggregatorValues.java      |   52 +
 .../apache/beam/sdk/runners/DirectPipeline.java |   56 +
 .../sdk/runners/DirectPipelineRegistrar.java    |   55 +
 .../beam/sdk/runners/DirectPipelineRunner.java  | 1322 ++++++++
 .../apache/beam/sdk/runners/PipelineRunner.java |   78 +
 .../sdk/runners/PipelineRunnerRegistrar.java    |   42 +
 .../sdk/runners/RecordingPipelineVisitor.java   |   55 +
 .../beam/sdk/runners/TransformHierarchy.java    |  106 +
 .../beam/sdk/runners/TransformTreeNode.java     |  254 ++
 .../inprocess/AbstractModelEnforcement.java     |   38 +
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 +
 .../inprocess/BoundedReadEvaluatorFactory.java  |  156 +
 .../sdk/runners/inprocess/BundleFactory.java    |   49 +
 .../CachedThreadPoolExecutorServiceFactory.java |   44 +
 .../beam/sdk/runners/inprocess/Clock.java       |   30 +
 .../runners/inprocess/CompletionCallback.java   |   36 +
 .../ConsumerTrackingPipelineVisitor.java        |  173 +
 .../inprocess/EmptyTransformEvaluator.java      |   50 +
 .../EncodabilityEnforcementFactory.java         |   71 +
 .../sdk/runners/inprocess/EvaluatorKey.java     |   55 +
 .../inprocess/ExecutorServiceFactory.java       |   33 +
 .../ExecutorServiceParallelExecutor.java        |  478 +++
 .../inprocess/FlattenEvaluatorFactory.java      |   85 +
 .../runners/inprocess/ForwardingPTransform.java |   56 +
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  274 ++
 .../ImmutabilityCheckingBundleFactory.java      |  132 +
 .../ImmutabilityEnforcementFactory.java         |  104 +
 .../inprocess/InMemoryWatermarkManager.java     | 1314 ++++++++
 .../sdk/runners/inprocess/InProcessBundle.java  |  124 +
 .../inprocess/InProcessBundleFactory.java       |  157 +
 .../inprocess/InProcessBundleOutputManager.java |   51 +
 .../sdk/runners/inprocess/InProcessCreate.java  |  236 ++
 .../inprocess/InProcessEvaluationContext.java   |  412 +++
 .../inprocess/InProcessExecutionContext.java    |  107 +
 .../runners/inprocess/InProcessExecutor.java    |   48 +
 .../inprocess/InProcessPipelineOptions.java     |  101 +
 .../inprocess/InProcessPipelineRunner.java      |  366 +++
 .../runners/inprocess/InProcessRegistrar.java   |   55 +
 .../inprocess/InProcessSideInputContainer.java  |  233 ++
 .../inprocess/InProcessTimerInternals.java      |   84 +
 .../inprocess/InProcessTransformResult.java     |   77 +
 .../inprocess/KeyedPValueTrackingVisitor.java   |   96 +
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 +
 .../inprocess/ModelEnforcementFactory.java      |   30 +
 .../sdk/runners/inprocess/NanosOffsetClock.java |   59 +
 .../inprocess/PTransformOverrideFactory.java    |   33 +
 .../inprocess/ParDoInProcessEvaluator.java      |  120 +
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   92 +
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   89 +
 .../PassthroughTransformEvaluator.java          |   49 +
 .../runners/inprocess/ShardControlledWrite.java |   81 +
 .../beam/sdk/runners/inprocess/StepAndKey.java  |   71 +
 .../runners/inprocess/StepTransformResult.java  |  165 +
 .../inprocess/TextIOShardedWriteFactory.java    |   78 +
 .../runners/inprocess/TransformEvaluator.java   |   46 +
 .../inprocess/TransformEvaluatorFactory.java    |   44 +
 .../inprocess/TransformEvaluatorRegistry.java   |   77 +
 .../runners/inprocess/TransformExecutor.java    |  176 +
 .../inprocess/TransformExecutorService.java     |   35 +
 .../inprocess/TransformExecutorServices.java    |  154 +
 .../UnboundedReadEvaluatorFactory.java          |  177 +
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 +
 .../inprocess/WatermarkCallbackExecutor.java    |  146 +
 .../inprocess/WindowEvaluatorFactory.java       |  131 +
 .../apache/beam/sdk/runners/package-info.java   |   34 +
 .../beam/sdk/runners/worker/IsmFormat.java      |  950 ++++++
 .../beam/sdk/runners/worker/package-info.java   |   25 +
 .../beam/sdk/testing/CoderProperties.java       |  351 ++
 .../org/apache/beam/sdk/testing/PAssert.java    |  826 +++++
 .../beam/sdk/testing/RunnableOnService.java     |   34 +
 .../beam/sdk/testing/SerializableMatcher.java   |   36 +
 .../beam/sdk/testing/SerializableMatchers.java  | 1182 +++++++
 .../beam/sdk/testing/SourceTestUtils.java       |  676 ++++
 .../apache/beam/sdk/testing/TestPipeline.java   |  206 ++
 .../beam/sdk/testing/WindowFnTestUtils.java     |  326 ++
 .../apache/beam/sdk/testing/package-info.java   |   22 +
 .../apache/beam/sdk/transforms/Aggregator.java  |   79 +
 .../sdk/transforms/AggregatorRetriever.java     |   36 +
 .../beam/sdk/transforms/AppliedPTransform.java  |  102 +
 .../sdk/transforms/ApproximateQuantiles.java    |  768 +++++
 .../beam/sdk/transforms/ApproximateUnique.java  |  421 +++
 .../org/apache/beam/sdk/transforms/Combine.java | 2244 +++++++++++++
 .../beam/sdk/transforms/CombineFnBase.java      |  286 ++
 .../apache/beam/sdk/transforms/CombineFns.java  | 1011 ++++++
 .../beam/sdk/transforms/CombineWithContext.java |  279 ++
 .../org/apache/beam/sdk/transforms/Count.java   |  193 ++
 .../org/apache/beam/sdk/transforms/Create.java  |  428 +++
 .../org/apache/beam/sdk/transforms/DoFn.java    |  565 ++++
 .../beam/sdk/transforms/DoFnReflector.java      |  670 ++++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  497 +++
 .../beam/sdk/transforms/DoFnWithContext.java    |  417 +++
 .../org/apache/beam/sdk/transforms/Filter.java  |  235 ++
 .../beam/sdk/transforms/FlatMapElements.java    |  146 +
 .../org/apache/beam/sdk/transforms/Flatten.java |  220 ++
 .../apache/beam/sdk/transforms/GroupByKey.java  |  276 ++
 .../transforms/IntraBundleParallelization.java  |  349 ++
 .../org/apache/beam/sdk/transforms/Keys.java    |   69 +
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   74 +
 .../apache/beam/sdk/transforms/MapElements.java |  113 +
 .../org/apache/beam/sdk/transforms/Max.java     |  256 ++
 .../org/apache/beam/sdk/transforms/Mean.java    |  204 ++
 .../org/apache/beam/sdk/transforms/Min.java     |  256 ++
 .../apache/beam/sdk/transforms/PTransform.java  |  325 ++
 .../org/apache/beam/sdk/transforms/ParDo.java   | 1323 ++++++++
 .../apache/beam/sdk/transforms/Partition.java   |  174 +
 .../beam/sdk/transforms/RemoveDuplicates.java   |  159 +
 .../org/apache/beam/sdk/transforms/Sample.java  |  248 ++
 .../sdk/transforms/SerializableComparator.java  |   29 +
 .../sdk/transforms/SerializableFunction.java    |   32 +
 .../beam/sdk/transforms/SimpleFunction.java     |   55 +
 .../org/apache/beam/sdk/transforms/Sum.java     |  189 ++
 .../org/apache/beam/sdk/transforms/Top.java     |  561 ++++
 .../org/apache/beam/sdk/transforms/Values.java  |   69 +
 .../org/apache/beam/sdk/transforms/View.java    |  471 +++
 .../apache/beam/sdk/transforms/WithKeys.java    |  141 +
 .../beam/sdk/transforms/WithTimestamps.java     |  130 +
 .../org/apache/beam/sdk/transforms/Write.java   |   29 +
 .../sdk/transforms/display/DisplayData.java     |  902 ++++++
 .../sdk/transforms/display/HasDisplayData.java  |   55 +
 .../beam/sdk/transforms/join/CoGbkResult.java   |  466 +++
 .../sdk/transforms/join/CoGbkResultSchema.java  |  135 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |  212 ++
 .../transforms/join/KeyedPCollectionTuple.java  |  248 ++
 .../beam/sdk/transforms/join/RawUnionValue.java |   51 +
 .../beam/sdk/transforms/join/UnionCoder.java    |  148 +
 .../beam/sdk/transforms/join/package-info.java  |   22 +
 .../beam/sdk/transforms/package-info.java       |   43 +
 .../beam/sdk/transforms/windowing/AfterAll.java |  115 +
 .../windowing/AfterDelayFromFirstElement.java   |  324 ++
 .../sdk/transforms/windowing/AfterEach.java     |  133 +
 .../sdk/transforms/windowing/AfterFirst.java    |  118 +
 .../sdk/transforms/windowing/AfterPane.java     |  143 +
 .../windowing/AfterProcessingTime.java          |   96 +
 .../AfterSynchronizedProcessingTime.java        |   77 +
 .../transforms/windowing/AfterWatermark.java    |  397 +++
 .../sdk/transforms/windowing/BoundedWindow.java |   47 +
 .../transforms/windowing/CalendarWindows.java   |  349 ++
 .../transforms/windowing/DefaultTrigger.java    |   94 +
 .../sdk/transforms/windowing/FixedWindows.java  |  117 +
 .../sdk/transforms/windowing/GlobalWindow.java  |   69 +
 .../sdk/transforms/windowing/GlobalWindows.java |   64 +
 .../transforms/windowing/IntervalWindow.java    |  202 ++
 .../transforms/windowing/InvalidWindows.java    |   88 +
 .../MergeOverlappingIntervalWindows.java        |   87 +
 .../windowing/NonMergingWindowFn.java           |   36 +
 .../transforms/windowing/OrFinallyTrigger.java  |  103 +
 .../sdk/transforms/windowing/OutputTimeFn.java  |  321 ++
 .../sdk/transforms/windowing/OutputTimeFns.java |  170 +
 .../beam/sdk/transforms/windowing/PaneInfo.java |  386 +++
 .../windowing/PartitioningWindowFn.java         |   62 +
 .../sdk/transforms/windowing/Repeatedly.java    |   98 +
 .../beam/sdk/transforms/windowing/Sessions.java |  113 +
 .../transforms/windowing/SlidingWindows.java    |  215 ++
 .../beam/sdk/transforms/windowing/Trigger.java  |  536 ++++
 .../transforms/windowing/TriggerBuilder.java    |   29 +
 .../beam/sdk/transforms/windowing/Window.java   |  654 ++++
 .../beam/sdk/transforms/windowing/WindowFn.java |  223 ++
 .../sdk/transforms/windowing/package-info.java  |   50 +
 .../apache/beam/sdk/util/ActiveWindowSet.java   |  184 ++
 .../org/apache/beam/sdk/util/ApiSurface.java    |  643 ++++
 .../beam/sdk/util/AppEngineEnvironment.java     |   62 +
 .../apache/beam/sdk/util/AppliedCombineFn.java  |  133 +
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 +
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   68 +
 ...AttemptAndTimeBoundedExponentialBackOff.java |  169 +
 .../util/AttemptBoundedExponentialBackOff.java  |   84 +
 .../org/apache/beam/sdk/util/AvroUtils.java     |  345 ++
 .../beam/sdk/util/BaseExecutionContext.java     |  156 +
 .../beam/sdk/util/BatchTimerInternals.java      |  140 +
 .../apache/beam/sdk/util/BigQueryServices.java  |   62 +
 .../beam/sdk/util/BigQueryServicesImpl.java     |  197 ++
 .../beam/sdk/util/BigQueryTableInserter.java    |  460 +++
 .../beam/sdk/util/BigQueryTableRowIterator.java |  472 +++
 .../org/apache/beam/sdk/util/BitSetCoder.java   |   61 +
 .../BufferedElementCountingOutputStream.java    |  186 ++
 .../apache/beam/sdk/util/CloudKnownType.java    |  139 +
 .../org/apache/beam/sdk/util/CloudObject.java   |  185 ++
 .../org/apache/beam/sdk/util/CoderUtils.java    |  329 ++
 .../beam/sdk/util/CombineContextFactory.java    |  109 +
 .../org/apache/beam/sdk/util/CombineFnUtil.java |  205 ++
 .../apache/beam/sdk/util/CounterAggregator.java |   97 +
 .../apache/beam/sdk/util/CredentialFactory.java |   30 +
 .../org/apache/beam/sdk/util/Credentials.java   |  194 ++
 .../beam/sdk/util/DataflowReleaseInfo.java      |   88 +
 .../sdk/util/DirectModeExecutionContext.java    |  132 +
 .../beam/sdk/util/DirectSideInputReader.java    |   75 +
 .../java/org/apache/beam/sdk/util/DoFnInfo.java |   68 +
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   62 +
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  560 ++++
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  144 +
 .../apache/beam/sdk/util/ExecutableTrigger.java |  159 +
 .../apache/beam/sdk/util/ExecutionContext.java  |  103 +
 .../sdk/util/ExposedByteArrayInputStream.java   |   52 +
 .../sdk/util/ExposedByteArrayOutputStream.java  |  118 +
 .../beam/sdk/util/FileIOChannelFactory.java     |  136 +
 .../apache/beam/sdk/util/FinishedTriggers.java  |   44 +
 .../beam/sdk/util/FinishedTriggersBitSet.java   |   67 +
 .../beam/sdk/util/FinishedTriggersSet.java      |   73 +
 .../beam/sdk/util/GcpCredentialFactory.java     |   47 +
 .../beam/sdk/util/GcsIOChannelFactory.java      |   87 +
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  412 +++
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |  105 +
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   59 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 +
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  246 ++
 .../apache/beam/sdk/util/IOChannelFactory.java  |  102 +
 .../apache/beam/sdk/util/IOChannelUtils.java    |  205 ++
 .../beam/sdk/util/IllegalMutationException.java |   53 +
 .../apache/beam/sdk/util/InstanceBuilder.java   |  271 ++
 .../util/IntervalBoundedExponentialBackOff.java |   94 +
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |   43 +
 .../beam/sdk/util/KeyedWorkItemCoder.java       |  123 +
 .../apache/beam/sdk/util/KeyedWorkItems.java    |  123 +
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  147 +
 .../beam/sdk/util/MapAggregatorValues.java      |   50 +
 .../beam/sdk/util/MergingActiveWindowSet.java   |  576 ++++
 .../org/apache/beam/sdk/util/MimeTypes.java     |   24 +
 .../apache/beam/sdk/util/MutationDetector.java  |   32 +
 .../apache/beam/sdk/util/MutationDetectors.java |  181 ++
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 +
 .../sdk/util/NonMergingActiveWindowSet.java     |   93 +
 .../beam/sdk/util/NoopCredentialFactory.java    |   40 +
 .../apache/beam/sdk/util/NoopPathValidator.java |   49 +
 .../beam/sdk/util/NullSideInputReader.java      |   63 +
 .../apache/beam/sdk/util/OutputReference.java   |   43 +
 .../beam/sdk/util/PCollectionViewWindow.java    |   68 +
 .../apache/beam/sdk/util/PCollectionViews.java  |  428 +++
 .../java/org/apache/beam/sdk/util/PTuple.java   |  161 +
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  154 +
 .../org/apache/beam/sdk/util/PathValidator.java |   48 +
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  149 +
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |  260 ++
 .../org/apache/beam/sdk/util/PropertyNames.java |  108 +
 .../apache/beam/sdk/util/RandomAccessData.java  |  355 +++
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  130 +
 .../beam/sdk/util/ReduceFnContextFactory.java   |  497 +++
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  905 ++++++
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |   47 +
 .../org/apache/beam/sdk/util/Reshuffle.java     |  147 +
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |   63 +
 .../sdk/util/RetryHttpRequestInitializer.java   |  251 ++
 .../apache/beam/sdk/util/SerializableUtils.java |  161 +
 .../org/apache/beam/sdk/util/Serializer.java    |  146 +
 .../sdk/util/ShardingWritableByteChannel.java   |  119 +
 .../apache/beam/sdk/util/SideInputReader.java   |   48 +
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   56 +
 .../org/apache/beam/sdk/util/StreamUtils.java   |   69 +
 .../org/apache/beam/sdk/util/StringUtils.java   |  244 ++
 .../java/org/apache/beam/sdk/util/Structs.java  |  385 +++
 .../beam/sdk/util/SystemDoFnInternal.java       |   39 +
 .../apache/beam/sdk/util/SystemReduceFn.java    |  135 +
 .../apache/beam/sdk/util/TestCredential.java    |   52 +
 .../org/apache/beam/sdk/util/TimeDomain.java    |   43 +
 .../java/org/apache/beam/sdk/util/TimeUtil.java |  166 +
 .../apache/beam/sdk/util/TimerInternals.java    |  271 ++
 .../java/org/apache/beam/sdk/util/Timers.java   |   60 +
 .../org/apache/beam/sdk/util/Transport.java     |  152 +
 .../beam/sdk/util/TriggerContextFactory.java    |  513 +++
 .../org/apache/beam/sdk/util/TriggerRunner.java |  234 ++
 .../beam/sdk/util/UnownedInputStream.java       |   76 +
 .../beam/sdk/util/UnownedOutputStream.java      |   56 +
 .../sdk/util/UploadIdResponseInterceptor.java   |   62 +
 .../apache/beam/sdk/util/UserCodeException.java |   95 +
 .../apache/beam/sdk/util/ValueWithRecordId.java |  156 +
 .../java/org/apache/beam/sdk/util/Values.java   |   89 +
 .../java/org/apache/beam/sdk/util/VarInt.java   |  116 +
 .../org/apache/beam/sdk/util/WatermarkHold.java |  477 +++
 .../java/org/apache/beam/sdk/util/Weighted.java |   28 +
 .../org/apache/beam/sdk/util/WeightedValue.java |   46 +
 .../org/apache/beam/sdk/util/WindowTracing.java |   37 +
 .../org/apache/beam/sdk/util/WindowedValue.java |  723 +++++
 .../beam/sdk/util/WindowingInternals.java       |   83 +
 .../apache/beam/sdk/util/WindowingStrategy.java |  268 ++
 .../java/org/apache/beam/sdk/util/ZipFiles.java |  295 ++
 .../apache/beam/sdk/util/common/Counter.java    | 1106 +++++++
 .../beam/sdk/util/common/CounterProvider.java   |   27 +
 .../apache/beam/sdk/util/common/CounterSet.java |  178 ++
 .../util/common/ElementByteSizeObservable.java  |   42 +
 .../ElementByteSizeObservableIterable.java      |   64 +
 .../ElementByteSizeObservableIterator.java      |   37 +
 .../util/common/ElementByteSizeObserver.java    |   93 +
 .../beam/sdk/util/common/PeekingReiterator.java |   99 +
 .../beam/sdk/util/common/ReflectHelpers.java    |  211 ++
 .../apache/beam/sdk/util/common/Reiterable.java |   28 +
 .../apache/beam/sdk/util/common/Reiterator.java |   40 +
 .../beam/sdk/util/common/package-info.java      |   19 +
 .../sdk/util/common/worker/StateSampler.java    |  367 +++
 .../sdk/util/common/worker/package-info.java    |   19 +
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java |  620 ++++
 .../beam/sdk/util/gcsfs/package-info.java       |   19 +
 .../org/apache/beam/sdk/util/package-info.java  |   19 +
 .../util/state/AccumulatorCombiningState.java   |   53 +
 .../apache/beam/sdk/util/state/BagState.java    |   28 +
 .../beam/sdk/util/state/CombiningState.java     |   42 +
 .../CopyOnAccessInMemoryStateInternals.java     |  457 +++
 .../sdk/util/state/InMemoryStateInternals.java  |  416 +++
 .../sdk/util/state/MergingStateAccessor.java    |   41 +
 .../beam/sdk/util/state/ReadableState.java      |   55 +
 .../org/apache/beam/sdk/util/state/State.java   |   32 +
 .../beam/sdk/util/state/StateAccessor.java      |   37 +
 .../beam/sdk/util/state/StateContext.java       |   43 +
 .../beam/sdk/util/state/StateContexts.java      |  109 +
 .../beam/sdk/util/state/StateInternals.java     |   57 +
 .../beam/sdk/util/state/StateMerging.java       |  257 ++
 .../beam/sdk/util/state/StateNamespace.java     |   56 +
 .../sdk/util/state/StateNamespaceForTest.java   |   65 +
 .../beam/sdk/util/state/StateNamespaces.java    |  280 ++
 .../apache/beam/sdk/util/state/StateTable.java  |   92 +
 .../apache/beam/sdk/util/state/StateTag.java    |   98 +
 .../apache/beam/sdk/util/state/StateTags.java   |  582 ++++
 .../apache/beam/sdk/util/state/ValueState.java  |   37 +
 .../beam/sdk/util/state/WatermarkHoldState.java |   44 +
 .../java/org/apache/beam/sdk/values/KV.java     |  133 +
 .../java/org/apache/beam/sdk/values/PBegin.java |   88 +
 .../org/apache/beam/sdk/values/PCollection.java |  251 ++
 .../apache/beam/sdk/values/PCollectionList.java |  240 ++
 .../beam/sdk/values/PCollectionTuple.java       |  266 ++
 .../apache/beam/sdk/values/PCollectionView.java |   65 +
 .../java/org/apache/beam/sdk/values/PDone.java  |   48 +
 .../java/org/apache/beam/sdk/values/PInput.java |   57 +
 .../org/apache/beam/sdk/values/POutput.java     |   77 +
 .../beam/sdk/values/POutputValueBase.java       |  103 +
 .../java/org/apache/beam/sdk/values/PValue.java |   39 +
 .../org/apache/beam/sdk/values/PValueBase.java  |  157 +
 .../beam/sdk/values/TimestampedValue.java       |  156 +
 .../org/apache/beam/sdk/values/TupleTag.java    |  199 ++
 .../apache/beam/sdk/values/TupleTagList.java    |  150 +
 .../apache/beam/sdk/values/TypeDescriptor.java  |  352 ++
 .../org/apache/beam/sdk/values/TypedPValue.java |  198 ++
 .../apache/beam/sdk/values/package-info.java    |   53 +
 .../main/proto/proto2_coder_test_messages.proto |    2 +-
 .../google/cloud/dataflow/sdk/sdk.properties    |    5 -
 .../org/apache/beam/sdk/sdk.properties          |    5 +
 .../cloud/dataflow/sdk/DataflowMatchers.java    |   66 -
 .../google/cloud/dataflow/sdk/PipelineTest.java |  297 --
 .../google/cloud/dataflow/sdk/TestUtils.java    |  214 --
 .../cloud/dataflow/sdk/WindowMatchers.java      |  138 -
 .../dataflow/sdk/coders/AvroCoderTest.java      |  755 -----
 .../sdk/coders/BigEndianIntegerCoderTest.java   |   91 -
 .../sdk/coders/BigEndianLongCoderTest.java      |   95 -
 .../dataflow/sdk/coders/ByteArrayCoderTest.java |  145 -
 .../dataflow/sdk/coders/ByteCoderTest.java      |   92 -
 .../sdk/coders/ByteStringCoderTest.java         |  122 -
 .../dataflow/sdk/coders/CoderFactoriesTest.java |  101 -
 .../dataflow/sdk/coders/CoderProvidersTest.java |   72 -
 .../dataflow/sdk/coders/CoderRegistryTest.java  |  522 ---
 .../cloud/dataflow/sdk/coders/CoderTest.java    |   80 -
 .../sdk/coders/CollectionCoderTest.java         |   94 -
 .../dataflow/sdk/coders/CustomCoderTest.java    |  136 -
 .../dataflow/sdk/coders/DefaultCoderTest.java   |  129 -
 .../dataflow/sdk/coders/DelegateCoderTest.java  |  142 -
 .../dataflow/sdk/coders/DoubleCoderTest.java    |   97 -
 .../dataflow/sdk/coders/DurationCoderTest.java  |   87 -
 .../dataflow/sdk/coders/EntityCoderTest.java    |  109 -
 .../dataflow/sdk/coders/InstantCoderTest.java   |  117 -
 .../dataflow/sdk/coders/IterableCoderTest.java  |  110 -
 .../dataflow/sdk/coders/JAXBCoderTest.java      |  156 -
 .../cloud/dataflow/sdk/coders/KvCoderTest.java  |  119 -
 .../dataflow/sdk/coders/ListCoderTest.java      |  135 -
 .../cloud/dataflow/sdk/coders/MapCoderTest.java |  107 -
 .../dataflow/sdk/coders/NullableCoderTest.java  |  140 -
 .../sdk/coders/PrintBase64Encodings.java        |   82 -
 .../dataflow/sdk/coders/Proto2CoderTest.java    |  146 -
 .../sdk/coders/SerializableCoderTest.java       |  223 --
 .../cloud/dataflow/sdk/coders/SetCoderTest.java |   87 -
 .../dataflow/sdk/coders/StandardCoderTest.java  |  177 -
 .../sdk/coders/StringDelegateCoderTest.java     |   73 -
 .../sdk/coders/StringUtf8CoderTest.java         |   81 -
 .../sdk/coders/StructuralByteArrayTest.java     |   41 -
 .../sdk/coders/TableRowJsonCoderTest.java       |   87 -
 .../sdk/coders/TextualIntegerCoderTest.java     |   91 -
 .../dataflow/sdk/coders/VarIntCoderTest.java    |   92 -
 .../dataflow/sdk/coders/VarLongCoderTest.java   |   95 -
 .../sdk/coders/protobuf/ProtoCoderTest.java     |  183 --
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |  196 --
 .../sdk/io/AvroIOGeneratedClassTest.java        |  375 ---
 .../cloud/dataflow/sdk/io/AvroIOTest.java       |  259 --
 .../cloud/dataflow/sdk/io/AvroSourceTest.java   |  693 ----
 .../cloud/dataflow/sdk/io/BigQueryIOTest.java   |  606 ----
 .../io/BoundedReadFromUnboundedSourceTest.java  |  133 -
 .../dataflow/sdk/io/CompressedSourceTest.java   |  477 ---
 .../dataflow/sdk/io/CountingInputTest.java      |  152 -
 .../dataflow/sdk/io/CountingSourceTest.java     |  290 --
 .../cloud/dataflow/sdk/io/DatastoreIOTest.java  |  588 ----
 .../dataflow/sdk/io/FileBasedSinkTest.java      |  513 ---
 .../dataflow/sdk/io/FileBasedSourceTest.java    |  917 ------
 .../dataflow/sdk/io/OffsetBasedSourceTest.java  |  281 --
 .../cloud/dataflow/sdk/io/PubsubIOTest.java     |  234 --
 .../google/cloud/dataflow/sdk/io/ReadTest.java  |  145 -
 .../cloud/dataflow/sdk/io/TextIOTest.java       |  517 ---
 .../google/cloud/dataflow/sdk/io/WriteTest.java |  344 --
 .../cloud/dataflow/sdk/io/XmlSinkTest.java      |  236 --
 .../cloud/dataflow/sdk/io/XmlSourceTest.java    |  825 -----
 .../sdk/io/bigtable/BigtableIOTest.java         |  689 ----
 .../range/ByteKeyRangeEstimateFractionTest.java |   70 -
 .../range/ByteKeyRangeInterpolateKeyTest.java   |   74 -
 .../dataflow/sdk/io/range/ByteKeyRangeTest.java |  397 ---
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |  120 -
 .../dataflow/sdk/io/range/ByteKeyTest.java      |  179 --
 .../sdk/io/range/OffsetRangeTrackerTest.java    |  187 --
 .../com/google/cloud/dataflow/sdk/io/user.avsc  |   10 -
 .../dataflow/sdk/options/GcpOptionsTest.java    |  125 -
 .../sdk/options/GoogleApiDebugOptionsTest.java  |  147 -
 .../sdk/options/PipelineOptionsFactoryTest.java | 1150 -------
 .../sdk/options/PipelineOptionsTest.java        |  127 -
 .../options/PipelineOptionsValidatorTest.java   |  311 --
 .../sdk/options/ProxyInvocationHandlerTest.java |  692 ----
 .../AggregatorPipelineExtractorTest.java        |  229 --
 .../runners/DirectPipelineRegistrarTest.java    |   70 -
 .../sdk/runners/DirectPipelineRunnerTest.java   |  211 --
 .../sdk/runners/DirectPipelineTest.java         |   35 -
 .../sdk/runners/PipelineRunnerTest.java         |   81 -
 .../dataflow/sdk/runners/TransformTreeTest.java |  195 --
 .../runners/dataflow/TestCountingSource.java    |  247 --
 .../AvroIOShardedWriteFactoryTest.java          |  112 -
 .../BoundedReadEvaluatorFactoryTest.java        |  290 --
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 --
 .../EncodabilityEnforcementFactoryTest.java     |  262 --
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 -
 .../inprocess/ForwardingPTransformTest.java     |  102 -
 .../GroupByKeyEvaluatorFactoryTest.java         |  182 --
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 --
 .../ImmutabilityEnforcementFactoryTest.java     |  127 -
 .../inprocess/InMemoryWatermarkManagerTest.java | 1124 -------
 .../inprocess/InProcessBundleFactoryTest.java   |  197 --
 .../runners/inprocess/InProcessBundleTest.java  |  145 -
 .../runners/inprocess/InProcessCreateTest.java  |  235 --
 .../InProcessEvaluationContextTest.java         |  535 ----
 .../InProcessPipelineRegistrarTest.java         |   73 -
 .../inprocess/InProcessPipelineRunnerTest.java  |   79 -
 .../InProcessSideInputContainerTest.java        |  372 ---
 .../inprocess/InProcessTimerInternalsTest.java  |  133 -
 .../KeyedPValueTrackingVisitorTest.java         |  191 --
 .../sdk/runners/inprocess/MockClock.java        |   62 -
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 ---
 .../ParDoSingleEvaluatorFactoryTest.java        |  325 --
 .../TextIOShardedWriteFactoryTest.java          |  112 -
 .../TransformExecutorServicesTest.java          |  136 -
 .../inprocess/TransformExecutorTest.java        |  537 ----
 .../UnboundedReadEvaluatorFactoryTest.java      |  333 --
 .../inprocess/ViewEvaluatorFactoryTest.java     |  101 -
 .../WatermarkCallbackExecutorTest.java          |  128 -
 .../inprocess/WindowEvaluatorFactoryTest.java   |  221 --
 .../sdk/testing/CoderPropertiesTest.java        |  215 --
 .../sdk/testing/DataflowJUnitTestRunner.java    |  129 -
 .../dataflow/sdk/testing/ExpectedLogs.java      |  307 --
 .../dataflow/sdk/testing/ExpectedLogsTest.java  |  154 -
 .../sdk/testing/FastNanoClockAndSleeper.java    |   48 -
 .../testing/FastNanoClockAndSleeperTest.java    |   48 -
 .../cloud/dataflow/sdk/testing/PAssertTest.java |  309 --
 .../sdk/testing/PCollectionViewTesting.java     |  296 --
 .../sdk/testing/ResetDateTimeProvider.java      |   42 -
 .../sdk/testing/ResetDateTimeProviderTest.java  |   56 -
 .../sdk/testing/RestoreSystemProperties.java    |   50 -
 .../testing/RestoreSystemPropertiesTest.java    |   51 -
 .../sdk/testing/SerializableMatchersTest.java   |  166 -
 .../sdk/testing/SystemNanoTimeSleeper.java      |   69 -
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |   54 -
 .../dataflow/sdk/testing/TestPipelineTest.java  |  103 -
 .../transforms/ApproximateQuantilesTest.java    |  298 --
 .../sdk/transforms/ApproximateUniqueTest.java   |  292 --
 .../dataflow/sdk/transforms/CombineFnsTest.java |  415 ---
 .../dataflow/sdk/transforms/CombineTest.java    | 1138 -------
 .../dataflow/sdk/transforms/CountTest.java      |  122 -
 .../dataflow/sdk/transforms/CreateTest.java     |  241 --
 .../sdk/transforms/DoFnContextTest.java         |   69 -
 .../DoFnDelegatingAggregatorTest.java           |  144 -
 .../sdk/transforms/DoFnReflectorTest.java       |  494 ---
 .../cloud/dataflow/sdk/transforms/DoFnTest.java |  202 --
 .../dataflow/sdk/transforms/DoFnTesterTest.java |  255 --
 .../sdk/transforms/DoFnWithContextTest.java     |  221 --
 .../dataflow/sdk/transforms/FilterTest.java     |  161 -
 .../sdk/transforms/FlatMapElementsTest.java     |  125 -
 .../dataflow/sdk/transforms/FlattenTest.java    |  370 ---
 .../dataflow/sdk/transforms/GroupByKeyTest.java |  377 ---
 .../IntraBundleParallelizationTest.java         |  251 --
 .../cloud/dataflow/sdk/transforms/KeysTest.java |   84 -
 .../dataflow/sdk/transforms/KvSwapTest.java     |   92 -
 .../sdk/transforms/MapElementsTest.java         |  135 -
 .../cloud/dataflow/sdk/transforms/MaxTest.java  |   67 -
 .../cloud/dataflow/sdk/transforms/MeanTest.java |   73 -
 .../cloud/dataflow/sdk/transforms/MinTest.java  |   67 -
 .../cloud/dataflow/sdk/transforms/NoOpDoFn.java |  144 -
 .../dataflow/sdk/transforms/PTransformTest.java |   42 -
 .../dataflow/sdk/transforms/ParDoTest.java      | 1542 ---------
 .../dataflow/sdk/transforms/PartitionTest.java  |  141 -
 .../sdk/transforms/RemoveDuplicatesTest.java    |  132 -
 .../dataflow/sdk/transforms/SampleTest.java     |  261 --
 .../sdk/transforms/SimpleStatsFnsTest.java      |  130 -
 .../cloud/dataflow/sdk/transforms/SumTest.java  |   68 -
 .../cloud/dataflow/sdk/transforms/TopTest.java  |  260 --
 .../dataflow/sdk/transforms/ValuesTest.java     |   94 -
 .../cloud/dataflow/sdk/transforms/ViewTest.java | 1423 ---------
 .../dataflow/sdk/transforms/WithKeysTest.java   |  128 -
 .../sdk/transforms/WithTimestampsTest.java      |  211 --
 .../transforms/display/DisplayDataMatchers.java |  321 --
 .../display/DisplayDataMatchersTest.java        |  157 -
 .../sdk/transforms/display/DisplayDataTest.java |  955 ------
 .../transforms/join/CoGbkResultCoderTest.java   |   86 -
 .../sdk/transforms/join/CoGbkResultTest.java    |  125 -
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  508 ---
 .../sdk/transforms/join/UnionCoderTest.java     |   49 -
 .../sdk/transforms/windowing/AfterAllTest.java  |  152 -
 .../sdk/transforms/windowing/AfterEachTest.java |  123 -
 .../transforms/windowing/AfterFirstTest.java    |  176 -
 .../sdk/transforms/windowing/AfterPaneTest.java |  127 -
 .../windowing/AfterProcessingTimeTest.java      |  158 -
 .../AfterSynchronizedProcessingTimeTest.java    |  122 -
 .../windowing/AfterWatermarkTest.java           |  339 --
 .../windowing/CalendarWindowsTest.java          |  261 --
 .../windowing/DefaultTriggerTest.java           |  177 -
 .../transforms/windowing/FixedWindowsTest.java  |  125 -
 .../windowing/IntervalWindowTest.java           |   95 -
 .../windowing/OrFinallyTriggerTest.java         |  210 --
 .../sdk/transforms/windowing/PaneInfoTest.java  |   77 -
 .../transforms/windowing/RepeatedlyTest.java    |  212 --
 .../sdk/transforms/windowing/SessionsTest.java  |  157 -
 .../windowing/SlidingWindowsTest.java           |  194 --
 .../sdk/transforms/windowing/TriggerTest.java   |  119 -
 .../sdk/transforms/windowing/WindowTest.java    |  227 --
 .../sdk/transforms/windowing/WindowingTest.java |  245 --
 .../cloud/dataflow/sdk/util/ApiSurfaceTest.java |  188 --
 ...mptAndTimeBoundedExponentialBackOffTest.java |  213 --
 .../AttemptBoundedExponentialBackOffTest.java   |   86 -
 .../cloud/dataflow/sdk/util/AvroUtilsTest.java  |  226 --
 .../sdk/util/BatchTimerInternalsTest.java       |  118 -
 .../sdk/util/BigQueryServicesImplTest.java      |  262 --
 .../sdk/util/BigQueryTableInserterTest.java     |  304 --
 .../sdk/util/BigQueryTableRowIteratorTest.java  |  256 --
 .../dataflow/sdk/util/BigQueryUtilTest.java     |  480 ---
 ...BufferedElementCountingOutputStreamTest.java |  208 --
 .../cloud/dataflow/sdk/util/CoderUtilsTest.java |  230 --
 .../dataflow/sdk/util/CombineFnUtilTest.java    |  104 -
 .../sdk/util/CounterAggregatorTest.java         |  254 --
 .../sdk/util/ExecutableTriggerTest.java         |  129 -
 .../util/ExposedByteArrayInputStreamTest.java   |   80 -
 .../util/ExposedByteArrayOutputStreamTest.java  |  247 --
 .../sdk/util/FileIOChannelFactoryTest.java      |  227 --
 .../sdk/util/FinishedTriggersBitSetTest.java    |   55 -
 .../sdk/util/FinishedTriggersProperties.java    |  110 -
 .../sdk/util/FinishedTriggersSetTest.java       |   61 -
 .../sdk/util/GcsIOChannelFactoryTest.java       |   44 -
 .../cloud/dataflow/sdk/util/GcsUtilTest.java    |  492 ---
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  719 -----
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  112 -
 .../dataflow/sdk/util/IOChannelUtilsTest.java   |   95 -
 .../dataflow/sdk/util/InstanceBuilderTest.java  |  116 -
 .../IntervalBoundedExponentialBackOffTest.java  |  100 -
 .../sdk/util/KeyedWorkItemCoderTest.java        |   63 -
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 -
 .../sdk/util/MergingActiveWindowSetTest.java    |  174 -
 .../sdk/util/MutationDetectorsTest.java         |  149 -
 .../cloud/dataflow/sdk/util/PTupleTest.java     |   41 -
 .../dataflow/sdk/util/RandomAccessDataTest.java |  207 --
 .../dataflow/sdk/util/ReduceFnRunnerTest.java   | 1205 -------
 .../cloud/dataflow/sdk/util/ReduceFnTester.java |  779 -----
 .../cloud/dataflow/sdk/util/ReshuffleTest.java  |  209 --
 .../dataflow/sdk/util/ReshuffleTriggerTest.java |   60 -
 .../util/RetryHttpRequestInitializerTest.java   |  297 --
 .../sdk/util/SerializableUtilsTest.java         |  166 -
 .../cloud/dataflow/sdk/util/SerializerTest.java |  163 -
 .../dataflow/sdk/util/SimpleDoFnRunnerTest.java |   87 -
 .../dataflow/sdk/util/StreamUtilsTest.java      |   73 -
 .../dataflow/sdk/util/StringUtilsTest.java      |  146 -
 .../cloud/dataflow/sdk/util/StructsTest.java    |  207 --
 .../cloud/dataflow/sdk/util/TimeUtilTest.java   |   74 -
 .../dataflow/sdk/util/TimerInternalsTest.java   |   54 -
 .../cloud/dataflow/sdk/util/TriggerTester.java  |  584 ----
 .../sdk/util/UnownedInputStreamTest.java        |   77 -
 .../sdk/util/UnownedOutputStreamTest.java       |   58 -
 .../util/UploadIdResponseInterceptorTest.java   |  101 -
 .../sdk/util/UserCodeExceptionTest.java         |  176 -
 .../cloud/dataflow/sdk/util/VarIntTest.java     |  278 --
 .../dataflow/sdk/util/WindowedValueTest.java    |   58 -
 .../cloud/dataflow/sdk/util/ZipFilesTest.java   |  312 --
 .../sdk/util/common/CounterSetTest.java         |  226 --
 .../dataflow/sdk/util/common/CounterTest.java   |  590 ----
 .../sdk/util/common/CounterTestUtils.java       |   57 -
 .../sdk/util/common/ReflectHelpersTest.java     |  128 -
 .../dataflow/sdk/util/gcsfs/GcsPathTest.java    |  334 --
 .../CopyOnAccessInMemoryStateInternalsTest.java |  555 ----
 .../util/state/InMemoryStateInternalsTest.java  |  350 --
 .../sdk/util/state/StateNamespacesTest.java     |  131 -
 .../dataflow/sdk/util/state/StateTagTest.java   |  175 -
 .../cloud/dataflow/sdk/values/KVTest.java       |  113 -
 .../sdk/values/PCollectionListTest.java         |   48 -
 .../sdk/values/PCollectionTupleTest.java        |   94 -
 .../cloud/dataflow/sdk/values/PDoneTest.java    |  103 -
 .../cloud/dataflow/sdk/values/TupleTagTest.java |   88 -
 .../dataflow/sdk/values/TypeDescriptorTest.java |  194 --
 .../dataflow/sdk/values/TypedPValueTest.java    |  165 -
 .../org/apache/beam/sdk/DataflowMatchers.java   |   66 +
 .../java/org/apache/beam/sdk/PipelineTest.java  |  298 ++
 .../java/org/apache/beam/sdk/TestUtils.java     |  214 ++
 .../org/apache/beam/sdk/WindowMatchers.java     |  138 +
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  755 +++++
 .../sdk/coders/BigEndianIntegerCoderTest.java   |   91 +
 .../beam/sdk/coders/BigEndianLongCoderTest.java |   95 +
 .../beam/sdk/coders/ByteArrayCoderTest.java     |  145 +
 .../apache/beam/sdk/coders/ByteCoderTest.java   |   92 +
 .../beam/sdk/coders/ByteStringCoderTest.java    |  123 +
 .../beam/sdk/coders/CoderFactoriesTest.java     |  101 +
 .../beam/sdk/coders/CoderProvidersTest.java     |   72 +
 .../beam/sdk/coders/CoderRegistryTest.java      |  523 +++
 .../org/apache/beam/sdk/coders/CoderTest.java   |   79 +
 .../beam/sdk/coders/CollectionCoderTest.java    |   94 +
 .../apache/beam/sdk/coders/CustomCoderTest.java |  136 +
 .../beam/sdk/coders/DefaultCoderTest.java       |  130 +
 .../beam/sdk/coders/DelegateCoderTest.java      |  143 +
 .../apache/beam/sdk/coders/DoubleCoderTest.java |   97 +
 .../beam/sdk/coders/DurationCoderTest.java      |   88 +
 .../apache/beam/sdk/coders/EntityCoderTest.java |  110 +
 .../beam/sdk/coders/InstantCoderTest.java       |  118 +
 .../beam/sdk/coders/IterableCoderTest.java      |  110 +
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |  157 +
 .../org/apache/beam/sdk/coders/KvCoderTest.java |  120 +
 .../apache/beam/sdk/coders/ListCoderTest.java   |  135 +
 .../apache/beam/sdk/coders/MapCoderTest.java    |  108 +
 .../beam/sdk/coders/NullableCoderTest.java      |  141 +
 .../beam/sdk/coders/PrintBase64Encodings.java   |   83 +
 .../apache/beam/sdk/coders/Proto2CoderTest.java |  148 +
 .../beam/sdk/coders/SerializableCoderTest.java  |  223 ++
 .../apache/beam/sdk/coders/SetCoderTest.java    |   87 +
 .../beam/sdk/coders/StandardCoderTest.java      |  177 +
 .../sdk/coders/StringDelegateCoderTest.java     |   73 +
 .../beam/sdk/coders/StringUtf8CoderTest.java    |   81 +
 .../sdk/coders/StructuralByteArrayTest.java     |   41 +
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |   88 +
 .../sdk/coders/TextualIntegerCoderTest.java     |   91 +
 .../apache/beam/sdk/coders/VarIntCoderTest.java |   91 +
 .../beam/sdk/coders/VarLongCoderTest.java       |   95 +
 .../sdk/coders/protobuf/ProtoCoderTest.java     |  184 ++
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |  198 ++
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  375 +++
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  260 ++
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  694 ++++
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  607 ++++
 .../io/BoundedReadFromUnboundedSourceTest.java  |  133 +
 .../beam/sdk/io/CompressedSourceTest.java       |  478 +++
 .../apache/beam/sdk/io/CountingInputTest.java   |  152 +
 .../apache/beam/sdk/io/CountingSourceTest.java  |  290 ++
 .../org/apache/beam/sdk/io/DatastoreIOTest.java |  590 ++++
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  513 +++
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  919 ++++++
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  282 ++
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  234 ++
 .../java/org/apache/beam/sdk/io/ReadTest.java   |  145 +
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  519 +++
 .../java/org/apache/beam/sdk/io/WriteTest.java  |  345 ++
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  237 ++
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  827 +++++
 .../beam/sdk/io/bigtable/BigtableIOTest.java    |  692 ++++
 .../range/ByteKeyRangeEstimateFractionTest.java |   70 +
 .../range/ByteKeyRangeInterpolateKeyTest.java   |   74 +
 .../beam/sdk/io/range/ByteKeyRangeTest.java     |  397 +++
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |  120 +
 .../apache/beam/sdk/io/range/ByteKeyTest.java   |  179 ++
 .../sdk/io/range/OffsetRangeTrackerTest.java    |  187 ++
 .../test/java/org/apache/beam/sdk/io/user.avsc  |   10 +
 .../apache/beam/sdk/options/GcpOptionsTest.java |  125 +
 .../sdk/options/GoogleApiDebugOptionsTest.java  |  148 +
 .../sdk/options/PipelineOptionsFactoryTest.java | 1150 +++++++
 .../beam/sdk/options/PipelineOptionsTest.java   |  127 +
 .../options/PipelineOptionsValidatorTest.java   |  311 ++
 .../sdk/options/ProxyInvocationHandlerTest.java |  692 ++++
 .../AggregatorPipelineExtractorTest.java        |  229 ++
 .../runners/DirectPipelineRegistrarTest.java    |   71 +
 .../sdk/runners/DirectPipelineRunnerTest.java   |  212 ++
 .../beam/sdk/runners/DirectPipelineTest.java    |   35 +
 .../beam/sdk/runners/PipelineRunnerTest.java    |   81 +
 .../beam/sdk/runners/TransformTreeTest.java     |  195 ++
 .../runners/dataflow/TestCountingSource.java    |  247 ++
 .../AvroIOShardedWriteFactoryTest.java          |  112 +
 .../BoundedReadEvaluatorFactoryTest.java        |  290 ++
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ++
 .../EncodabilityEnforcementFactoryTest.java     |  262 ++
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 +
 .../inprocess/ForwardingPTransformTest.java     |  102 +
 .../GroupByKeyEvaluatorFactoryTest.java         |  183 ++
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 ++
 .../ImmutabilityEnforcementFactoryTest.java     |  126 +
 .../inprocess/InMemoryWatermarkManagerTest.java | 1125 +++++++
 .../inprocess/InProcessBundleFactoryTest.java   |  198 ++
 .../runners/inprocess/InProcessBundleTest.java  |  145 +
 .../runners/inprocess/InProcessCreateTest.java  |  236 ++
 .../InProcessEvaluationContextTest.java         |  536 ++++
 .../InProcessPipelineRegistrarTest.java         |   74 +
 .../inprocess/InProcessPipelineRunnerTest.java  |   78 +
 .../InProcessSideInputContainerTest.java        |  373 +++
 .../inprocess/InProcessTimerInternalsTest.java  |  133 +
 .../KeyedPValueTrackingVisitorTest.java         |  192 ++
 .../beam/sdk/runners/inprocess/MockClock.java   |   62 +
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 +++
 .../ParDoSingleEvaluatorFactoryTest.java        |  324 ++
 .../TextIOShardedWriteFactoryTest.java          |  112 +
 .../TransformExecutorServicesTest.java          |  136 +
 .../inprocess/TransformExecutorTest.java        |  538 ++++
 .../UnboundedReadEvaluatorFactoryTest.java      |  334 ++
 .../inprocess/ViewEvaluatorFactoryTest.java     |  101 +
 .../WatermarkCallbackExecutorTest.java          |  128 +
 .../inprocess/WindowEvaluatorFactoryTest.java   |  222 ++
 .../beam/sdk/testing/CoderPropertiesTest.java   |  216 ++
 .../sdk/testing/DataflowJUnitTestRunner.java    |  130 +
 .../apache/beam/sdk/testing/ExpectedLogs.java   |  307 ++
 .../beam/sdk/testing/ExpectedLogsTest.java      |  154 +
 .../sdk/testing/FastNanoClockAndSleeper.java    |   48 +
 .../testing/FastNanoClockAndSleeperTest.java    |   48 +
 .../apache/beam/sdk/testing/PAssertTest.java    |  310 ++
 .../sdk/testing/PCollectionViewTesting.java     |  297 ++
 .../beam/sdk/testing/ResetDateTimeProvider.java |   42 +
 .../sdk/testing/ResetDateTimeProviderTest.java  |   56 +
 .../sdk/testing/RestoreSystemProperties.java    |   50 +
 .../testing/RestoreSystemPropertiesTest.java    |   51 +
 .../sdk/testing/SerializableMatchersTest.java   |  168 +
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |   69 +
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |   55 +
 .../beam/sdk/testing/TestPipelineTest.java      |  103 +
 .../transforms/ApproximateQuantilesTest.java    |  299 ++
 .../sdk/transforms/ApproximateUniqueTest.java   |  293 ++
 .../beam/sdk/transforms/CombineFnsTest.java     |  416 +++
 .../apache/beam/sdk/transforms/CombineTest.java | 1141 +++++++
 .../apache/beam/sdk/transforms/CountTest.java   |  123 +
 .../apache/beam/sdk/transforms/CreateTest.java  |  242 ++
 .../beam/sdk/transforms/DoFnContextTest.java    |   69 +
 .../DoFnDelegatingAggregatorTest.java           |  144 +
 .../beam/sdk/transforms/DoFnReflectorTest.java  |  494 +++
 .../apache/beam/sdk/transforms/DoFnTest.java    |  202 ++
 .../beam/sdk/transforms/DoFnTesterTest.java     |  254 ++
 .../sdk/transforms/DoFnWithContextTest.java     |  221 ++
 .../apache/beam/sdk/transforms/FilterTest.java  |  161 +
 .../sdk/transforms/FlatMapElementsTest.java     |  126 +
 .../apache/beam/sdk/transforms/FlattenTest.java |  371 +++
 .../beam/sdk/transforms/GroupByKeyTest.java     |  378 +++
 .../IntraBundleParallelizationTest.java         |  252 ++
 .../apache/beam/sdk/transforms/KeysTest.java    |   84 +
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   92 +
 .../beam/sdk/transforms/MapElementsTest.java    |  135 +
 .../org/apache/beam/sdk/transforms/MaxTest.java |   68 +
 .../apache/beam/sdk/transforms/MeanTest.java    |   75 +
 .../org/apache/beam/sdk/transforms/MinTest.java |   68 +
 .../apache/beam/sdk/transforms/NoOpDoFn.java    |  144 +
 .../beam/sdk/transforms/PTransformTest.java     |   42 +
 .../apache/beam/sdk/transforms/ParDoTest.java   | 1544 +++++++++
 .../beam/sdk/transforms/PartitionTest.java      |  141 +
 .../sdk/transforms/RemoveDuplicatesTest.java    |  132 +
 .../apache/beam/sdk/transforms/SampleTest.java  |  263 ++
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |  130 +
 .../org/apache/beam/sdk/transforms/SumTest.java |   69 +
 .../org/apache/beam/sdk/transforms/TopTest.java |  260 ++
 .../apache/beam/sdk/transforms/ValuesTest.java  |   94 +
 .../apache/beam/sdk/transforms/ViewTest.java    | 1424 +++++++++
 .../beam/sdk/transforms/WithKeysTest.java       |  128 +
 .../beam/sdk/transforms/WithTimestampsTest.java |  211 ++
 .../transforms/display/DisplayDataMatchers.java |  322 ++
 .../display/DisplayDataMatchersTest.java        |  158 +
 .../sdk/transforms/display/DisplayDataTest.java |  958 ++++++
 .../transforms/join/CoGbkResultCoderTest.java   |   87 +
 .../sdk/transforms/join/CoGbkResultTest.java    |  125 +
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  509 +++
 .../sdk/transforms/join/UnionCoderTest.java     |   49 +
 .../sdk/transforms/windowing/AfterAllTest.java  |  152 +
 .../sdk/transforms/windowing/AfterEachTest.java |  123 +
 .../transforms/windowing/AfterFirstTest.java    |  176 +
 .../sdk/transforms/windowing/AfterPaneTest.java |  127 +
 .../windowing/AfterProcessingTimeTest.java      |  158 +
 .../AfterSynchronizedProcessingTimeTest.java    |  122 +
 .../windowing/AfterWatermarkTest.java           |  339 ++
 .../windowing/CalendarWindowsTest.java          |  262 ++
 .../windowing/DefaultTriggerTest.java           |  177 +
 .../transforms/windowing/FixedWindowsTest.java  |  126 +
 .../windowing/IntervalWindowTest.java           |   96 +
 .../windowing/OrFinallyTriggerTest.java         |  210 ++
 .../sdk/transforms/windowing/PaneInfoTest.java  |   77 +
 .../transforms/windowing/RepeatedlyTest.java    |  212 ++
 .../sdk/transforms/windowing/SessionsTest.java  |  158 +
 .../windowing/SlidingWindowsTest.java           |  195 ++
 .../sdk/transforms/windowing/TriggerTest.java   |  119 +
 .../sdk/transforms/windowing/WindowTest.java    |  227 ++
 .../sdk/transforms/windowing/WindowingTest.java |  245 ++
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  188 ++
 ...mptAndTimeBoundedExponentialBackOffTest.java |  214 ++
 .../AttemptBoundedExponentialBackOffTest.java   |   85 +
 .../org/apache/beam/sdk/util/AvroUtilsTest.java |  227 ++
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  118 +
 .../beam/sdk/util/BigQueryServicesImplTest.java |  262 ++
 .../sdk/util/BigQueryTableInserterTest.java     |  306 ++
 .../sdk/util/BigQueryTableRowIteratorTest.java  |  256 ++
 .../apache/beam/sdk/util/BigQueryUtilTest.java  |  481 +++
 ...BufferedElementCountingOutputStreamTest.java |  208 ++
 .../apache/beam/sdk/util/CoderUtilsTest.java    |  231 ++
 .../apache/beam/sdk/util/CombineFnUtilTest.java |  105 +
 .../beam/sdk/util/CounterAggregatorTest.java    |  256 ++
 .../beam/sdk/util/ExecutableTriggerTest.java    |  129 +
 .../util/ExposedByteArrayInputStreamTest.java   |   80 +
 .../util/ExposedByteArrayOutputStreamTest.java  |  247 ++
 .../beam/sdk/util/FileIOChannelFactoryTest.java |  227 ++
 .../sdk/util/FinishedTriggersBitSetTest.java    |   55 +
 .../sdk/util/FinishedTriggersProperties.java    |  110 +
 .../beam/sdk/util/FinishedTriggersSetTest.java  |   61 +
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |   44 +
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  493 +++
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  720 +++++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  112 +
 .../beam/sdk/util/IOChannelUtilsTest.java       |   95 +
 .../beam/sdk/util/InstanceBuilderTest.java      |  116 +
 .../IntervalBoundedExponentialBackOffTest.java  |  100 +
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |   64 +
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 +
 .../sdk/util/MergingActiveWindowSetTest.java    |  175 +
 .../beam/sdk/util/MutationDetectorsTest.java    |  149 +
 .../org/apache/beam/sdk/util/PTupleTest.java    |   41 +
 .../beam/sdk/util/RandomAccessDataTest.java     |  207 ++
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1207 +++++++
 .../apache/beam/sdk/util/ReduceFnTester.java    |  780 +++++
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  210 ++
 .../beam/sdk/util/ReshuffleTriggerTest.java     |   60 +
 .../util/RetryHttpRequestInitializerTest.java   |  297 ++
 .../beam/sdk/util/SerializableUtilsTest.java    |  167 +
 .../apache/beam/sdk/util/SerializerTest.java    |  163 +
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   86 +
 .../apache/beam/sdk/util/StreamUtilsTest.java   |   73 +
 .../apache/beam/sdk/util/StringUtilsTest.java   |  146 +
 .../org/apache/beam/sdk/util/StructsTest.java   |  207 ++
 .../org/apache/beam/sdk/util/TimeUtilTest.java  |   75 +
 .../beam/sdk/util/TimerInternalsTest.java       |   53 +
 .../org/apache/beam/sdk/util/TriggerTester.java |  586 ++++
 .../beam/sdk/util/UnownedInputStreamTest.java   |   76 +
 .../beam/sdk/util/UnownedOutputStreamTest.java  |   57 +
 .../util/UploadIdResponseInterceptorTest.java   |  101 +
 .../beam/sdk/util/UserCodeExceptionTest.java    |  175 +
 .../org/apache/beam/sdk/util/VarIntTest.java    |  278 ++
 .../apache/beam/sdk/util/WindowedValueTest.java |   58 +
 .../org/apache/beam/sdk/util/ZipFilesTest.java  |  312 ++
 .../beam/sdk/util/common/CounterSetTest.java    |  227 ++
 .../beam/sdk/util/common/CounterTest.java       |  591 ++++
 .../beam/sdk/util/common/CounterTestUtils.java  |   57 +
 .../sdk/util/common/ReflectHelpersTest.java     |  128 +
 .../apache/beam/sdk/util/gcsfs/GcsPathTest.java |  334 ++
 .../CopyOnAccessInMemoryStateInternalsTest.java |  555 ++++
 .../util/state/InMemoryStateInternalsTest.java  |  350 ++
 .../sdk/util/state/StateNamespacesTest.java     |  131 +
 .../beam/sdk/util/state/StateTagTest.java       |  175 +
 .../java/org/apache/beam/sdk/values/KVTest.java |  113 +
 .../beam/sdk/values/PCollectionListTest.java    |   48 +
 .../beam/sdk/values/PCollectionTupleTest.java   |   94 +
 .../org/apache/beam/sdk/values/PDoneTest.java   |  103 +
 .../apache/beam/sdk/values/TupleTagTest.java    |   88 +
 .../beam/sdk/values/TypeDescriptorTest.java     |  194 ++
 .../apache/beam/sdk/values/TypedPValueTest.java |  164 +
 .../PipelineOptionsFactoryJava8Test.java        |   92 -
 .../sdk/transforms/CombineJava8Test.java        |  134 -
 .../sdk/transforms/FilterJava8Test.java         |  119 -
 .../transforms/FlatMapElementsJava8Test.java    |   85 -
 .../sdk/transforms/MapElementsJava8Test.java    |   78 -
 .../sdk/transforms/PartitionJava8Test.java      |   75 -
 .../transforms/RemoveDuplicatesJava8Test.java   |  100 -
 .../sdk/transforms/WithKeysJava8Test.java       |   75 -
 .../sdk/transforms/WithTimestampsJava8Test.java |   67 -
 .../PipelineOptionsFactoryJava8Test.java        |   92 +
 .../beam/sdk/transforms/CombineJava8Test.java   |  134 +
 .../beam/sdk/transforms/FilterJava8Test.java    |  119 +
 .../transforms/FlatMapElementsJava8Test.java    |   86 +
 .../sdk/transforms/MapElementsJava8Test.java    |   78 +
 .../beam/sdk/transforms/PartitionJava8Test.java |   75 +
 .../transforms/RemoveDuplicatesJava8Test.java   |  101 +
 .../beam/sdk/transforms/WithKeysJava8Test.java  |   75 +
 .../sdk/transforms/WithTimestampsJava8Test.java |   67 +
 .../src/main/java/DebuggingWordCount.java       |   32 +-
 .../src/main/java/MinimalWordCount.java         |   18 +-
 .../src/main/java/WindowedWordCount.java        |   28 +-
 .../src/main/java/WordCount.java                |   34 +-
 .../java/common/DataflowExampleOptions.java     |    6 +-
 .../main/java/common/DataflowExampleUtils.java  |   22 +-
 .../common/ExampleBigQueryTableOptions.java     |   10 +-
 .../java/common/ExamplePubsubTopicOptions.java  |   10 +-
 .../main/java/common/PubsubFileInjector.java    |   20 +-
 .../src/test/java/WordCountTest.java            |   18 +-
 .../src/main/java/StarterPipeline.java          |   10 +-
 .../src/main/java/it/pkg/StarterPipeline.java   |   10 +-
 travis/test_wordcount.sh                        |    4 +-
 1801 files changed, 176475 insertions(+), 176001 deletions(-)
----------------------------------------------------------------------



[36/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java
deleted file mode 100644
index 2818251..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java
+++ /dev/null
@@ -1,3009 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximatePTransformName;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
-import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInEmptyWindows;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.services.clouddebugger.v2.Clouddebugger;
-import com.google.api.services.clouddebugger.v2.model.Debuggee;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.io.Write;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.JobSpecification;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.runners.dataflow.AssignWindows;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.util.DataflowTransport;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.PCollectionViews;
-import com.google.cloud.dataflow.sdk.util.PathValidator;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Reshuffle;
-import com.google.cloud.dataflow.sdk.util.SystemDoFnInternal;
-import com.google.cloud.dataflow.sdk.util.ValueWithRecordId;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-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;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.joda.time.DateTimeUtils;
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.format.DateTimeFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.Serializable;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to the Dataflow representation
- * using the {@link DataflowPipelineTranslator} and then submitting
- * them to a Dataflow service for execution.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code DataflowPipelineRunner}, the Google cloudservices account and the Google compute engine
- * service account of the GCP project running the Dataflow Job will need access to the corresponding
- * source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
-  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class);
-
-  /** Provided configuration options. */
-  private final DataflowPipelineOptions options;
-
-  /** Client for the Dataflow service. This is used to actually submit jobs. */
-  private final Dataflow dataflowClient;
-
-  /** Translator for this DataflowPipelineRunner, based on options. */
-  private final DataflowPipelineTranslator translator;
-
-  /** Custom transforms implementations. */
-  private final Map<Class<?>, Class<?>> overrides;
-
-  /** A set of user defined functions to invoke at different points in execution. */
-  private DataflowPipelineRunnerHooks hooks;
-
-  // Environment version information.
-  private static final String ENVIRONMENT_MAJOR_VERSION = "4";
-
-  // Default Docker container images that execute Dataflow worker harness, residing in Google
-  // Container Registry, separately for Batch and Streaming.
-  public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
-      = "dataflow.gcr.io/v1beta3/java-batch:1.5.1";
-  public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
-      = "dataflow.gcr.io/v1beta3/java-streaming:1.5.1";
-
-  // The limit of CreateJob request size.
-  private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
-
-  private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
-
-  /**
-   * Project IDs must contain lowercase letters, digits, or dashes.
-   * IDs must start with a letter and may not end with a dash.
-   * This regex isn't exact - this allows for patterns that would be rejected by
-   * the service, but this is sufficient for basic validation of project IDs.
-   */
-  public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
-
-  /**
-   * Construct a runner from the provided options.
-   *
-   * @param options Properties that configure the runner.
-   * @return The newly created runner.
-   */
-  public static DataflowPipelineRunner fromOptions(PipelineOptions options) {
-    // (Re-)register standard IO factories. Clobbers any prior credentials.
-    IOChannelUtils.registerStandardIOFactories(options);
-
-    DataflowPipelineOptions dataflowOptions =
-        PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
-    ArrayList<String> missing = new ArrayList<>();
-
-    if (dataflowOptions.getAppName() == null) {
-      missing.add("appName");
-    }
-    if (missing.size() > 0) {
-      throw new IllegalArgumentException(
-          "Missing required values: " + Joiner.on(',').join(missing));
-    }
-
-    PathValidator validator = dataflowOptions.getPathValidator();
-    Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
-        && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
-        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
-    if (dataflowOptions.getStagingLocation() != null) {
-      validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
-    }
-    if (dataflowOptions.getTempLocation() != null) {
-      validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
-    }
-    if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
-      dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
-    } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
-      try {
-        dataflowOptions.setStagingLocation(
-            IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
-      } catch (IOException e) {
-        throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
-            + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
-      }
-    }
-
-    if (dataflowOptions.getFilesToStage() == null) {
-      dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
-          DataflowPipelineRunner.class.getClassLoader()));
-      LOG.info("PipelineOptions.filesToStage was not specified. "
-          + "Defaulting to files from the classpath: will stage {} files. "
-          + "Enable logging at DEBUG level to see which files will be staged.",
-          dataflowOptions.getFilesToStage().size());
-      LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
-    }
-
-    // Verify jobName according to service requirements.
-    String jobName = dataflowOptions.getJobName().toLowerCase();
-    Preconditions.checkArgument(
-        jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
-        "JobName invalid; the name must consist of only the characters "
-            + "[-a-z0-9], starting with a letter and ending with a letter "
-            + "or number");
-
-    // Verify project
-    String project = dataflowOptions.getProject();
-    if (project.matches("[0-9]*")) {
-      throw new IllegalArgumentException("Project ID '" + project
-          + "' invalid. Please make sure you specified the Project ID, not project number.");
-    } else if (!project.matches(PROJECT_ID_REGEXP)) {
-      throw new IllegalArgumentException("Project ID '" + project
-          + "' invalid. Please make sure you specified the Project ID, not project description.");
-    }
-
-    DataflowPipelineDebugOptions debugOptions =
-        dataflowOptions.as(DataflowPipelineDebugOptions.class);
-    // Verify the number of worker threads is a valid value
-    if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
-      throw new IllegalArgumentException("Number of worker harness threads '"
-          + debugOptions.getNumberOfWorkerHarnessThreads()
-          + "' invalid. Please make sure the value is non-negative.");
-    }
-
-    return new DataflowPipelineRunner(dataflowOptions);
-  }
-
-  @VisibleForTesting protected DataflowPipelineRunner(DataflowPipelineOptions options) {
-    this.options = options;
-    this.dataflowClient = options.getDataflowClient();
-    this.translator = DataflowPipelineTranslator.fromOptions(options);
-    this.pcollectionsRequiringIndexedFormat = new HashSet<>();
-    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
-
-    if (options.isStreaming()) {
-      overrides = ImmutableMap.<Class<?>, Class<?>>builder()
-          .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class)
-          .put(Create.Values.class, StreamingCreate.class)
-          .put(View.AsMap.class, StreamingViewAsMap.class)
-          .put(View.AsMultimap.class, StreamingViewAsMultimap.class)
-          .put(View.AsSingleton.class, StreamingViewAsSingleton.class)
-          .put(View.AsList.class, StreamingViewAsList.class)
-          .put(View.AsIterable.class, StreamingViewAsIterable.class)
-          .put(Write.Bound.class, StreamingWrite.class)
-          .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class)
-          .put(Read.Unbounded.class, StreamingUnboundedRead.class)
-          .put(Read.Bounded.class, UnsupportedIO.class)
-          .put(AvroIO.Read.Bound.class, UnsupportedIO.class)
-          .put(AvroIO.Write.Bound.class, UnsupportedIO.class)
-          .put(BigQueryIO.Read.Bound.class, UnsupportedIO.class)
-          .put(TextIO.Read.Bound.class, UnsupportedIO.class)
-          .put(TextIO.Write.Bound.class, UnsupportedIO.class)
-          .put(Window.Bound.class, AssignWindows.class)
-          .build();
-    } else {
-      ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
-      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);
-      if (options.getExperiments() == null
-          || !options.getExperiments().contains("disable_ism_side_input")) {
-        builder.put(View.AsMap.class, BatchViewAsMap.class);
-        builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
-        builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
-        builder.put(View.AsList.class, BatchViewAsList.class);
-        builder.put(View.AsIterable.class, BatchViewAsIterable.class);
-      }
-      overrides = builder.build();
-    }
-  }
-
-  /**
-   * Applies the given transform to the input. For transforms with customized definitions
-   * for the Dataflow pipeline runner, the application is intercepted and modified here.
-   */
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-
-    if (Combine.GroupedValues.class.equals(transform.getClass())
-        || GroupByKey.class.equals(transform.getClass())) {
-
-      // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
-      // primitives. Returning a primitive output instead of the expanded definition
-      // signals to the translator that translation is necessary.
-      @SuppressWarnings("unchecked")
-      PCollection<?> pc = (PCollection<?>) input;
-      @SuppressWarnings("unchecked")
-      OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
-          pc.getPipeline(),
-          transform instanceof GroupByKey
-              ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
-              : pc.getWindowingStrategy(),
-          pc.isBounded());
-      return outputT;
-    } else if (Window.Bound.class.equals(transform.getClass())) {
-      /*
-       * TODO: make this the generic way overrides are applied (using super.apply() rather than
-       * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
-       * additional nodes into the graph.
-       */
-      // casting to wildcard
-      @SuppressWarnings("unchecked")
-      OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
-      return windowed;
-    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
-        && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform(input, Create.of());
-    } else if (overrides.containsKey(transform.getClass())) {
-      // It is the responsibility of whoever constructs overrides to ensure this is type safe.
-      @SuppressWarnings("unchecked")
-      Class<PTransform<InputT, OutputT>> transformClass =
-          (Class<PTransform<InputT, OutputT>>) transform.getClass();
-
-      @SuppressWarnings("unchecked")
-      Class<PTransform<InputT, OutputT>> customTransformClass =
-          (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
-
-      PTransform<InputT, OutputT> customTransform =
-          InstanceBuilder.ofType(customTransformClass)
-          .withArg(DataflowPipelineRunner.class, this)
-          .withArg(transformClass, transform)
-          .build();
-
-      return Pipeline.applyTransform(input, customTransform);
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-  private <T> PCollection<T> applyWindow(
-      Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
-    // types are matched at compile time
-    @SuppressWarnings("unchecked")
-    Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
-    @SuppressWarnings("unchecked")
-    PCollection<T> input = (PCollection<T>) initialInput;
-    return super.apply(new AssignWindows<>(transform), input);
-  }
-
-  private String debuggerMessage(String projectId, String uniquifier) {
-    return String.format("To debug your job, visit Google Cloud Debugger at: "
-        + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
-        projectId, uniquifier);
-  }
-
-  private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
-    if (!options.getEnableCloudDebugger()) {
-      return;
-    }
-
-    if (options.getDebuggee() != null) {
-      throw new RuntimeException("Should not specify the debuggee");
-    }
-
-    Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
-    Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
-    options.setDebuggee(debuggee);
-
-    System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
-  }
-
-  private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
-    RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
-    registerReq.setDebuggee(new Debuggee()
-        .setProject(options.getProject())
-        .setUniquifier(uniquifier)
-        .setDescription(uniquifier)
-        .setAgentVersion("google.com/cloud-dataflow-java/v1"));
-
-    try {
-      RegisterDebuggeeResponse registerResponse =
-          debuggerClient.controller().debuggees().register(registerReq).execute();
-      Debuggee debuggee = registerResponse.getDebuggee();
-      if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
-        throw new RuntimeException("Unable to register with the debugger: " +
-            debuggee.getStatus().getDescription().getFormat());
-      }
-
-      return debuggee;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to register with the debugger: ", e);
-    }
-  }
-
-  @Override
-  public DataflowPipelineJob run(Pipeline pipeline) {
-    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
-
-    LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
-        + "related to Google Compute Engine usage and other Google Cloud Services.");
-
-    List<DataflowPackage> packages = options.getStager().stageFiles();
-
-
-    // Set a unique client_request_id in the CreateJob request.
-    // This is used to ensure idempotence of job creation across retried
-    // attempts to create a job. Specifically, if the service returns a job with
-    // a different client_request_id, it means the returned one is a different
-    // job previously created with the same job name, and that the job creation
-    // has been effectively rejected. The SDK should return
-    // Error::Already_Exists to user in that case.
-    int randomNum = new Random().nextInt(9000) + 1000;
-    String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
-        .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
-
-    // Try to create a debuggee ID. This must happen before the job is translated since it may
-    // update the options.
-    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-    maybeRegisterDebuggee(dataflowOptions, requestId);
-
-    JobSpecification jobSpecification =
-        translator.translate(pipeline, this, packages);
-    Job newJob = jobSpecification.getJob();
-    newJob.setClientRequestId(requestId);
-
-    String version = DataflowReleaseInfo.getReleaseInfo().getVersion();
-    System.out.println("Dataflow SDK version: " + version);
-
-    newJob.getEnvironment().setUserAgent(DataflowReleaseInfo.getReleaseInfo());
-    // The Dataflow Service may write to the temporary directory directly, so
-    // must be verified.
-    if (!Strings.isNullOrEmpty(options.getTempLocation())) {
-      newJob.getEnvironment().setTempStoragePrefix(
-          dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
-    }
-    newJob.getEnvironment().setDataset(options.getTempDatasetId());
-    newJob.getEnvironment().setExperiments(options.getExperiments());
-
-    // Set the Docker container image that executes Dataflow worker harness, residing in Google
-    // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
-    String workerHarnessContainerImage =
-        options.as(DataflowPipelineWorkerPoolOptions.class)
-        .getWorkerHarnessContainerImage();
-    for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
-      workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
-    }
-
-    // Requirements about the service.
-    Map<String, Object> environmentVersion = new HashMap<>();
-    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
-    newJob.getEnvironment().setVersion(environmentVersion);
-    // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
-    // autoscale if specified.
-    String jobType = "JAVA_BATCH_AUTOSCALING";
-
-    if (options.isStreaming()) {
-      jobType = "STREAMING";
-    }
-    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
-
-    if (hooks != null) {
-      hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
-    }
-
-    if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
-      try (PrintWriter printWriter = new PrintWriter(
-          new File(options.getDataflowJobFile()))) {
-        String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
-        printWriter.print(workSpecJson);
-        LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
-      } catch (IllegalStateException ex) {
-        LOG.warn("Cannot translate workflow spec to json for debug.");
-      } catch (FileNotFoundException ex) {
-        LOG.warn("Cannot create workflow spec output file.");
-      }
-    }
-
-    String jobIdToUpdate = null;
-    if (options.getUpdate()) {
-      jobIdToUpdate = getJobIdFromName(options.getJobName());
-      newJob.setTransformNameMapping(options.getTransformNameMapping());
-      newJob.setReplaceJobId(jobIdToUpdate);
-    }
-    Job jobResult;
-    try {
-      jobResult = dataflowClient
-              .projects()
-              .jobs()
-              .create(options.getProject(), newJob)
-              .execute();
-    } catch (GoogleJsonResponseException e) {
-      String errorMessages = "Unexpected errors";
-      if (e.getDetails() != null) {
-        if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
-          errorMessages = "The size of the serialized JSON representation of the pipeline "
-              + "exceeds the allowable limit. "
-              + "For more information, please check the FAQ link below:\n"
-              + "https://cloud.google.com/dataflow/faq";
-        } else {
-          errorMessages = e.getDetails().getMessage();
-        }
-      }
-      throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to create a workflow job", e);
-    }
-
-    // Obtain all of the extractors from the PTransforms used in the pipeline so the
-    // DataflowPipelineJob has access to them.
-    AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
-    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
-        aggregatorExtractor.getAggregatorSteps();
-
-    DataflowAggregatorTransforms aggregatorTransforms =
-        new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
-
-    // Use a raw client for post-launch monitoring, as status calls may fail
-    // regularly and need not be retried automatically.
-    DataflowPipelineJob dataflowPipelineJob =
-        new DataflowPipelineJob(options.getProject(), jobResult.getId(),
-            DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
-
-    // If the service returned client request id, the SDK needs to compare it
-    // with the original id generated in the request, if they are not the same
-    // (i.e., the returned job is not created by this request), throw
-    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
-    // depending on whether this is a reload or not.
-    if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
-        && !jobResult.getClientRequestId().equals(requestId)) {
-      // If updating a job.
-      if (options.getUpdate()) {
-        throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
-            String.format("The job named %s with id: %s has already been updated into job id: %s "
-                + "and cannot be updated again.",
-                newJob.getName(), jobIdToUpdate, jobResult.getId()));
-      } else {
-        throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
-            String.format("There is already an active job named %s with id: %s. If you want "
-                + "to submit a second job, try again by setting a different name using --jobName.",
-                newJob.getName(), jobResult.getId()));
-      }
-    }
-
-    LOG.info("To access the Dataflow monitoring console, please navigate to {}",
-        MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
-    System.out.println("Submitted job: " + jobResult.getId());
-
-    LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
-        MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
-
-    return dataflowPipelineJob;
-  }
-
-  /**
-   * Returns the DataflowPipelineTranslator associated with this object.
-   */
-  public DataflowPipelineTranslator getTranslator() {
-    return translator;
-  }
-
-  /**
-   * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}.
-   */
-  @Experimental
-  public void setHooks(DataflowPipelineRunnerHooks hooks) {
-    this.hooks = hooks;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Outputs a warning about PCollection views without deterministic key coders. */
-  private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
-    // We need to wait till this point to determine the names of the transforms since only
-    // at this time do we know the hierarchy of the transforms otherwise we could
-    // have just recorded the full names during apply time.
-    if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
-      final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
-      pipeline.traverseTopologically(new PipelineVisitor() {
-        @Override
-        public void visitValue(PValue value, TransformTreeNode producer) {
-        }
-
-        @Override
-        public void visitTransform(TransformTreeNode node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-        }
-
-        @Override
-        public void enterCompositeTransform(TransformTreeNode node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-        }
-
-        @Override
-        public void leaveCompositeTransform(TransformTreeNode node) {
-        }
-      });
-
-      LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
-          + "because the key coder is not deterministic. Falling back to singleton implementation "
-          + "which may cause memory and/or performance problems. Future major versions of "
-          + "Dataflow will require deterministic key coders.",
-          ptransformViewNamesWithNonDeterministicKeyCoders);
-    }
-  }
-
-  /**
-   * Returns true if the passed in {@link PCollection} needs to be materialiazed using
-   * an indexed format.
-   */
-  boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
-    return pcollectionsRequiringIndexedFormat.contains(pcol);
-  }
-
-  /**
-   * Marks the passed in {@link PCollection} as requiring to be materialized using
-   * an indexed format.
-   */
-  private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
-    pcollectionsRequiringIndexedFormat.add(pcol);
-  }
-
-  /** A set of {@link View}s with non-deterministic key coders. */
-  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
-
-  /**
-   * Records that the {@link PTransform} requires a deterministic key coder.
-   */
-  private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
-    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
-  }
-
-  /**
-   * A {@link GroupByKey} transform for the {@link DataflowPipelineRunner} which sorts
-   * values using the secondary key {@code K2}.
-   *
-   * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
-   * the empty window. Care must be taken *afterwards* to either re-window
-   * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
-   * values being within a window.
-   */
-  static class GroupByKeyAndSortValuesOnly<K1, K2, V>
-      extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
-    private GroupByKeyAndSortValuesOnly() {
-    }
-
-    @Override
-    public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
-      PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
-          PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          IsBounded.BOUNDED);
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
-      rval.setCoder(
-          KvCoder.of(inputCoder.getKeyCoder(),
-          IterableCoder.of(inputCoder.getValueCoder())));
-      return rval;
-    }
-  }
-
-  /**
-   * A {@link PTransform} that groups the values by a hash of the window's byte representation
-   * and sorts the values using the windows byte representation.
-   */
-  private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
-      PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
-
-    /**
-     * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
-     * grouping by the hash of the window's byte representation and sorting the grouped values
-     * using the window's byte representation.
-     */
-    @SystemDoFnInternal
-    private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
-        extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
-
-      private final IsmRecordCoder<?> ismCoderForHash;
-      private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
-        this.ismCoderForHash = ismCoderForHash;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        @SuppressWarnings("unchecked")
-        W window = (W) c.window();
-        c.output(
-            KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
-                KV.of(window,
-                    WindowedValue.of(
-                        c.element(),
-                        c.timestamp(),
-                        c.window(),
-                        c.pane()))));
-      }
-    }
-
-    private final IsmRecordCoder<?> ismCoderForHash;
-    private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
-      this.ismCoderForHash = ismCoderForHash;
-    }
-
-    @Override
-    public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-      PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
-          input.apply(ParDo.of(
-              new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
-      rval.setCoder(
-          KvCoder.of(
-              VarIntCoder.of(),
-              KvCoder.of(windowCoder,
-                  FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
-      return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
-   * byte representation and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsSingleton<T>
-      extends PTransform<PCollection<T>, PCollectionView<T>> {
-
-    /**
-     * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
-     * <ul>
-     *   <li>Key 1: Window
-     *   <li>Value: Windowed value
-     * </ul>
-     */
-    static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                     IsmRecord<WindowedValue<T>>> {
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
-        while (iterator.hasNext()) {
-          KV<W, WindowedValue<T>> next = iterator.next();
-          c.output(
-              IsmRecord.of(
-                  ImmutableList.of(next.getKey()), next.getValue()));
-        }
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    private final View.AsSingleton<T> transform;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsSingleton(DataflowPipelineRunner runner, View.AsSingleton<T> transform) {
-      this.runner = runner;
-      this.transform = transform;
-    }
-
-    @Override
-    public PCollectionView<T> apply(PCollection<T> input) {
-      return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
-          runner,
-          input,
-          new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(),
-          transform.hasDefaultValue(),
-          transform.defaultValue(),
-          input.getCoder());
-    }
-
-    static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
-        applyForSingleton(
-            DataflowPipelineRunner runner,
-            PCollection<T> input,
-            DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                 IsmRecord<WindowedValue<FinalT>>> doFn,
-            boolean hasDefault,
-            FinalT defaultValue,
-            Coder<FinalT> defaultValueCoder) {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      PCollectionView<ViewT> view =
-          (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
-              input.getPipeline(),
-              (WindowingStrategy) input.getWindowingStrategy(),
-              hasDefault,
-              defaultValue,
-              defaultValueCoder);
-
-      IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
-          coderForSingleton(windowCoder, defaultValueCoder);
-
-      PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
-              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
-              .apply(ParDo.of(doFn));
-      reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-      return reifiedPerWindowAndSorted.apply(
-          CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsSingleton";
-    }
-
-    static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
-        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
-      return IsmRecordCoder.of(
-          1, // We hash using only the window
-          0, // There are no metadata records
-          ImmutableList.<Coder<?>>of(windowCoder),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsIterable View.AsIterable} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
-   * and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Key 2: Index offset within window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsIterable<T>
-      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsIterable(DataflowPipelineRunner runner, View.AsIterable<T> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
-      PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
-          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
-      return BatchViewAsList.applyForIterableLike(runner, input, view);
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsList View.AsList} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
-   * and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Key 2: Index offset within window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsList<T>
-      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
-     * global window. Each {@link IsmRecord} has
-     * <ul>
-     *   <li>Key 1: Global window</li>
-     *   <li>Key 2: Index offset within window</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     */
-    @SystemDoFnInternal
-    static class ToIsmRecordForGlobalWindowDoFn<T>
-        extends DoFn<T, IsmRecord<WindowedValue<T>>> {
-
-      long indexInBundle;
-      @Override
-      public void startBundle(Context c) throws Exception {
-        indexInBundle = 0;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(IsmRecord.of(
-            ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
-            WindowedValue.of(
-                c.element(),
-                c.timestamp(),
-                GlobalWindow.INSTANCE,
-                c.pane())));
-        indexInBundle += 1;
-      }
-    }
-
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
-     * to locate the window boundaries. The {@link IsmRecord} has:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Key 2: Index offset within window</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     */
-    @SystemDoFnInternal
-    static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                     IsmRecord<WindowedValue<T>>> {
-
-      private final Coder<W> windowCoder;
-      ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        long elementsInWindow = 0;
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
-          // Compare to see if this is a new window so we can reset the index counter i
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Reset i since we have a new window.
-            elementsInWindow = 0;
-          }
-          c.output(IsmRecord.of(
-              ImmutableList.of(value.getKey(), elementsInWindow),
-              value.getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          elementsInWindow += 1;
-        }
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsList(DataflowPipelineRunner runner, View.AsList<T> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<List<T>> apply(PCollection<T> input) {
-      PCollectionView<List<T>> view = PCollectionViews.listView(
-          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
-      return applyForIterableLike(runner, input, view);
-    }
-
-    static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
-        DataflowPipelineRunner runner,
-        PCollection<T> input,
-        PCollectionView<ViewT> view) {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
-
-      // If we are working in the global window, we do not need to do a GBK using the window
-      // as the key since all the elements of the input PCollection are already such.
-      // We just reify the windowed value while converting them to IsmRecords and generating
-      // an index based upon where we are within the bundle. Each bundle
-      // maps to one file exactly.
-      if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
-        PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
-            input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
-        reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-        runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-        return reifiedPerWindowAndSorted.apply(
-            CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
-      }
-
-      PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
-              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
-              .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
-      reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-      return reifiedPerWindowAndSorted.apply(
-          CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsList";
-    }
-
-    static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
-        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
-      // TODO: swap to use a variable length long coder which has values which compare
-      // the same as their byte representation compare lexicographically within the key coder
-      return IsmRecordCoder.of(
-          1, // We hash using only the window
-          0, // There are no metadata records
-          ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMap View.AsMap} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
-   * representation. Each record is structured as follows:
-   * <ul>
-   *   <li>Key 1: User key K</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: 0L (constant)</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   *
-   * <p>Alongside the data records, there are the following metadata records:
-   * <ul>
-   *   <li>Key 1: Metadata Key</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index [0, size of map]</li>
-   *   <li>Value: variable length long byte representation of size of map if index is 0,
-   *              otherwise the byte representation of a key</li>
-   * </ul>
-   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
-   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
-   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
-   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
-   * {@code [1, size of map]}.
-   *
-   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
-   * a warning to users to specify a deterministic key coder.
-   */
-  static class BatchViewAsMap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
-
-    /**
-     * A {@link DoFn} which groups elements by window boundaries. For each group,
-     * the group of elements is transformed into a {@link TransformedMap}.
-     * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
-     * and contains a function {@code WindowedValue<V> -> V}.
-     *
-     * <p>Outputs {@link IsmRecord}s having:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Value: Transformed map containing a transform that removes the encapsulation
-     *              of the window around each value,
-     *              {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
-     * </ul>
-     */
-    static class ToMapDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
-                     IsmRecord<WindowedValue<TransformedMap<K,
-                                             WindowedValue<V>,
-                                             V>>>> {
-
-      private final Coder<W> windowCoder;
-      ToMapDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c)
-          throws Exception {
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        Optional<W> previousWindow = Optional.absent();
-        Map<K, WindowedValue<V>> map = new HashMap<>();
-        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Construct the transformed map containing all the elements since we
-            // are at a window boundary.
-            c.output(IsmRecord.of(
-                ImmutableList.of(previousWindow.get()),
-                valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
-            map = new HashMap<>();
-          }
-
-          // Verify that the user isn't trying to insert the same key multiple times.
-          checkState(!map.containsKey(kv.getValue().getValue().getKey()),
-              "Multiple values [%s, %s] found for single key [%s] within window [%s].",
-              map.get(kv.getValue().getValue().getKey()),
-              kv.getValue().getValue().getValue(),
-              kv.getKey());
-          map.put(kv.getValue().getValue().getKey(),
-                  kv.getValue().withValue(kv.getValue().getValue().getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          previousWindow = Optional.of(kv.getKey());
-        }
-
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a transformed map containing all the elements since the last
-        // window boundary.
-        c.output(IsmRecord.of(
-            ImmutableList.of(previousWindow.get()),
-            valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsMap(DataflowPipelineRunner runner, View.AsMap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
-      return this.<BoundedWindow>applyInternal(input);
-    }
-
-    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
-        applyInternal(PCollection<KV<K, V>> input) {
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
-            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
-        return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
-      } catch (NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-
-        // Since the key coder is not deterministic, we convert the map into a singleton
-        // and return a singleton view equivalent.
-        return applyForSingletonFallback(input);
-      }
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsMap";
-    }
-
-    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
-        applyForSingletonFallback(PCollection<KV<K, V>> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      Coder<Function<WindowedValue<V>, V>> transformCoder =
-          (Coder) SerializableCoder.of(WindowedValueToValue.class);
-
-      Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
-          TransformedMapCoder.of(
-          transformCoder,
-          MapCoder.of(
-              inputCoder.getKeyCoder(),
-              FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
-
-      TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
-          WindowedValueToValue.<V>of(),
-          ImmutableMap.<K, WindowedValue<V>>of());
-
-      return BatchViewAsSingleton.<KV<K, V>,
-                                   TransformedMap<K, WindowedValue<V>, V>,
-                                   Map<K, V>,
-                                   W> applyForSingleton(
-          runner,
-          input,
-          new ToMapDoFn<K, V, W>(windowCoder),
-          true,
-          defaultValue,
-          finalValueCoder);
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMultimap View.AsMultimap} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
-   * representation. Each record is structured as follows:
-   * <ul>
-   *   <li>Key 1: User key K</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index offset for a given key and window.</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   *
-   * <p>Alongside the data records, there are the following metadata records:
-   * <ul>
-   *   <li>Key 1: Metadata Key</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index [0, size of map]</li>
-   *   <li>Value: variable length long byte representation of size of map if index is 0,
-   *              otherwise the byte representation of a key</li>
-   * </ul>
-   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
-   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
-   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
-   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
-   * {@code [1, size of map]}.
-   *
-   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
-   * a warning to users to specify a deterministic key coder.
-   */
-  static class BatchViewAsMultimap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
-    /**
-     * A {@link PTransform} that groups elements by the hash of window's byte representation
-     * if the input {@link PCollection} is not within the global window. Otherwise by the hash
-     * of the window and key's byte representation. This {@link PTransform} also sorts
-     * the values by the combination of the window and key's byte representations.
-     */
-    private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
-        extends PTransform<PCollection<KV<K, V>>,
-                           PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
-
-      @SystemDoFnInternal
-      private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
-          extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
-          implements DoFn.RequiresWindowAccess {
-
-        private final IsmRecordCoder<?> coder;
-        private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
-          this.coder = coder;
-        }
-
-        @Override
-        public void processElement(ProcessContext c) throws Exception {
-          @SuppressWarnings("unchecked")
-          W window = (W) c.window();
-
-          c.output(
-              KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
-                  KV.of(KV.of(c.element().getKey(), window),
-                      WindowedValue.of(
-                          c.element().getValue(),
-                          c.timestamp(),
-                          (BoundedWindow) window,
-                          c.pane()))));
-        }
-      }
-
-      private final IsmRecordCoder<?> coder;
-      public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
-        this.coder = coder;
-      }
-
-      @Override
-      public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
-          apply(PCollection<KV<K, V>> input) {
-
-        @SuppressWarnings("unchecked")
-        Coder<W> windowCoder = (Coder<W>)
-            input.getWindowingStrategy().getWindowFn().windowCoder();
-        @SuppressWarnings("unchecked")
-        KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
-        PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
-        keyedByHash = input.apply(
-            ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
-        keyedByHash.setCoder(
-            KvCoder.of(
-                VarIntCoder.of(),
-                KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
-                    FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
-        return keyedByHash.apply(
-            new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
-      }
-    }
-
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
-     * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Key 2: User key K</li>
-     *   <li>Key 3: Index offset for a given key and window.</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     *
-     * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
-     * and the unique key count per window to {@code outputForSize}.
-     *
-     * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
-     * throw an {@link IllegalStateException} if more than one key per window is found.
-     */
-    static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
-                     IsmRecord<WindowedValue<V>>> {
-
-      private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
-      private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
-      private final Coder<W> windowCoder;
-      private final Coder<K> keyCoder;
-      private final IsmRecordCoder<WindowedValue<V>> ismCoder;
-      private final boolean uniqueKeysExpected;
-      ToIsmRecordForMapLikeDoFn(
-          TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
-          TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
-          Coder<W> windowCoder,
-          Coder<K> keyCoder,
-          IsmRecordCoder<WindowedValue<V>> ismCoder,
-          boolean uniqueKeysExpected) {
-        this.outputForSize = outputForSize;
-        this.outputForEntrySet = outputForEntrySet;
-        this.windowCoder = windowCoder;
-        this.keyCoder = keyCoder;
-        this.ismCoder = ismCoder;
-        this.uniqueKeysExpected = uniqueKeysExpected;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        long currentKeyIndex = 0;
-        // We use one based indexing while counting
-        long currentUniqueKeyCounter = 1;
-        Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
-
-        KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
-        Object currentKeyStructuralValue =
-            keyCoder.structuralValue(currentValue.getKey().getKey());
-        Object currentWindowStructuralValue =
-            windowCoder.structuralValue(currentValue.getKey().getValue());
-
-        while (iterator.hasNext()) {
-          KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
-          Object nextKeyStructuralValue =
-              keyCoder.structuralValue(nextValue.getKey().getKey());
-          Object nextWindowStructuralValue =
-              windowCoder.structuralValue(nextValue.getKey().getValue());
-
-          outputDataRecord(c, currentValue, currentKeyIndex);
-
-          final long nextKeyIndex;
-          final long nextUniqueKeyCounter;
-
-          // Check to see if its a new window
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            // The next value is a new window, so we output for size the number of unique keys
-            // seen and the last key of the window. We also reset the next key index the unique
-            // key counter.
-            outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
-            outputMetadataRecordForEntrySet(c, currentValue);
-
-            nextKeyIndex = 0;
-            nextUniqueKeyCounter = 1;
-          } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
-            // It is a new key within the same window so output the key for the entry set,
-            // reset the key index and increase the count of unique keys seen within this window.
-            outputMetadataRecordForEntrySet(c, currentValue);
-
-            nextKeyIndex = 0;
-            nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
-          } else if (!uniqueKeysExpected) {
-            // It is not a new key so we don't have to output the number of elements in this
-            // window or increase the unique key counter. All we do is increase the key index.
-
-            nextKeyIndex = currentKeyIndex + 1;
-            nextUniqueKeyCounter = currentUniqueKeyCounter;
-          } else {
-            throw new IllegalStateException(String.format(
-                "Unique keys are expected but found key %s with values %s and %s in window %s.",
-                currentValue.getKey().getKey(),
-                currentValue.getValue().getValue(),
-                nextValue.getValue().getValue(),
-                currentValue.getKey().getValue()));
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-          currentKeyStructuralValue = nextKeyStructuralValue;
-          currentKeyIndex = nextKeyIndex;
-          currentUniqueKeyCounter = nextUniqueKeyCounter;
-        }
-
-        outputDataRecord(c, currentValue, currentKeyIndex);
-        outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a record with the number of unique keys seen.
-        outputMetadataRecordForEntrySet(c, currentValue);
-      }
-
-      /** This outputs the data record. */
-      private void outputDataRecord(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
-        IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
-            ImmutableList.of(
-                value.getKey().getKey(),
-                value.getKey().getValue(),
-                keyIndex),
-            value.getValue());
-        c.output(ismRecord);
-      }
-
-      /**
-       * This outputs records which will be used to compute the number of keys for a given window.
-       */
-      private void outputMetadataRecordForSize(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
-        c.sideOutput(outputForSize,
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
-                                                 value.getKey().getValue())),
-                KV.of(value.getKey().getValue(), uniqueKeyCount)));
-      }
-
-      /** This outputs records which will be used to construct the entry set. */
-      private void outputMetadataRecordForEntrySet(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
-        c.sideOutput(outputForEntrySet,
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
-                                                 value.getKey().getValue())),
-                KV.of(value.getKey().getValue(), value.getKey().getKey())));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
-       * <ul>
-       *   <li>Key 1: META key</li>
-       *   <li>Key 2: window</li>
-       *   <li>Key 3: 0L (constant)</li>
-       *   <li>Value: sum of values for window</li>
-       * </ul>
-       *
-       * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
-       * per window for map and multimap side inputs.
-       */
-    static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
-      private final Coder<W> windowCoder;
-      ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
-        KV<W, Long> currentValue = iterator.next();
-        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
-        long size = 0;
-        while (iterator.hasNext()) {
-          KV<W, Long> nextValue = iterator.next();
-          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
-          size += currentValue.getValue();
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            c.output(IsmRecord.<WindowedValue<V>>meta(
-                ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
-                CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
-            size = 0;
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-        }
-
-        size += currentValue.getValue();
-        // Output the final value since it is guaranteed to be on a window boundary.
-        c.output(IsmRecord.<WindowedValue<V>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
-       * <ul>
-       *   <li>Key 1: META key</li>
-       *   <li>Key 2: window</li>
-       *   <li>Key 3: index offset (1-based index)</li>
-       *   <li>Value: key</li>
-       * </ul>
-       *
-       * <p>This {@link DoFn} is meant to be used to output index to key records
-       * per window for map and multimap side inputs.
-       */
-    static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
-
-      private final Coder<K> keyCoder;
-      private final Coder<W> windowCoder;
-      ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
-        this.keyCoder = keyCoder;
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
-        KV<W, K> currentValue = iterator.next();
-        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
-        long elementsInWindow = 1;
-        while (iterator.hasNext()) {
-          KV<W, K> nextValue = iterator.next();
-          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
-          c.output(IsmRecord.<WindowedValue<V>>meta(
-              ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
-              CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
-          elementsInWindow += 1;
-
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            elementsInWindow = 1;
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-        }
-
-        // Output the final value since it is guaranteed to be on a window boundary.
-        c.output(IsmRecord.<WindowedValue<V>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
-            CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
-     * partition, the set of elements is transformed into a {@link TransformedMap}.
-     * The transformed {@code Map<K, Iterable<V>>} is backed by a
-     * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
-     * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
-     *
-     * <p>Outputs {@link IsmRecord}s having:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Value: Transformed map containing a transform that removes the encapsulation
-     *              of the window around each value,
-     *              {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
-     * </ul>
-     */
-    static class ToMultimapDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
-                     IsmRecord<WindowedValue<TransformedMap<K,
-                                                            Iterable<WindowedValue<V>>,
-                                                            Iterable<V>>>>> {
-
-      private final Coder<W> windowCoder;
-      ToMultimapDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c)
-          throws Exception {
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        Optional<W> previousWindow = Optional.absent();
-        Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
-        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Construct the transformed map containing all the elements since we
-            // are at a window boundary.
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
-            c.output(IsmRecord.<WindowedValue<TransformedMap<K,
-                                                             Iterable<WindowedValue<V>>,
-                                                             Iterable<V>>>>of(
-                ImmutableList.of(previousWindow.get()),
-                valueInEmptyWindows(
-                    new TransformedMap<>(
-                        IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
-            multimap = HashMultimap.create();
-          }
-
-          multimap.put(kv.getValue().getValue().getKey(),
-                       kv.getValue().withValue(kv.getValue().getValue().getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          previousWindow = Optional.of(kv.getKey());
-        }
-
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a transformed map containing all the elements since the last
-        // window boundary.
-        @SuppressWarnings({"unchecked", "rawtypes"})
-        Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
-        c.output(IsmRecord.<WindowedValue<TransformedMap<K,
-                                                         Iterable<WindowedValue<V>>,
-                                                         Iterable<V>>>>of(
-            ImmutableList.of(previousWindow.get()),
-            valueInEmptyWindows(
-                new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsMultimap(DataflowPipelineRunner runner, View.AsMultimap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-      return this.<BoundedWindow>applyInternal(input);
-    }
-
-    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
-        applyInternal(PCollection<KV<K, V>> input) {
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
-            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
-
-        return applyForMapLike(runner, input, view, false /* unique keys not expected */);
-      } catch (NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-
-        // Since the key coder is not deterministic, we convert the map into a singleton
-        // and return a singleton view equivalent.
-        return applyForSingletonFallback(input);
-      }
-    }
-
-    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
-        applyForSingletonFallback(PCollection<KV<K, V>> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
-          (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
-
-      Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
-          TransformedMapCoder.of(
-          transformCoder,
-          MapCoder.of(
-              inputCoder.getKeyCoder(),
-              IterableCoder.of(
-                  FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
-      TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
-          new TransformedMap<>(
-              IterableWithWindowedValuesToIterable.<V>of(),
-              ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
-
-      return BatchViewAsSingleton.<KV<K, V>,
-                                   TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
-                                   Map<K, Iterable<V>>,
-                                   W> applyForSingleton(
-          runner,
-          input,
-          new ToMultimapDoFn<K, V, W>(windowCoder),
-          true,
-          defaultValue,
-          finalValueCoder);
-    }
-
-    private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
-        DataflowPipelineRunner runner,
-        PCollection<KV<K, V>> input,
-        PCollectionView<ViewT> view,
-        boolean uniqueKeysExpected) throws NonDeterministicException {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      // If our key coder is deterministic, we can use the key portion of each KV
-      // part of a composite key containing the window , key and index.
-      inputCoder.getKeyCoder().verifyDeterministic();
-
-      IsmRecordCoder<WindowedValue<V>> ismCoder =
-          coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
-
-      // Create the various output tags representing the main output containing the data stream
-      // and the side outputs containing the metadata about the size and entry set.
-      TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
-      TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
-      TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
-
-      // Process all the elements grouped by key hash, and sorted by key and then window
-      // outputting to all the outputs defined above.
-      PCollectionTuple outputTuple = input
-           .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
-           .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
-                   outputForSizeTag, outputForEntrySetTag,
-                   windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
-                       .withOutputTags(mainOutputTag,
-                                       TupleTagList.of(
-                                           ImmutableList.<TupleTag<?>>of(outputForSizeTag,
-                                                                         outputForEntrySetTag))));
-
-      // Set the coder on the main data output.
-      PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
-          outputTuple.get(mainOutputTag);
-      perHashWithReifiedWindows.setCoder(ismCoder);
-
-      // Set the coder on the metadata output for size and process the entries
-      // producing a [META, Window, 0L] record per window storing the number of unique keys
-      // for each window.
-      PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
-      outputForSize.setCoder(
-          KvCoder.of(VarIntCoder.of(),
-                     KvCoder.of(windowCoder, VarLongCoder.of())));
-      PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
-          .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
-          .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
-      windowMapSizeMetadata.setCoder(ismCoder);
-
-      // Set the coder on the metadata output destined to build the entry set and process the
-      // entries producing a [META, Window, Index] record per window key pair storing the key.
-      PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
-          outputTuple.get(outputForEntrySetTag);
-      outputForEntrySet.setCoder(
-          KvCoder.of(VarIntCoder.of(),
-                     KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
-      PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
-          .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
-          .apply(ParDo.of(
-              new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
-      windowMapKeysMetadata.setCoder(ismCoder);
-
-      // Set that all these outputs should be materialized using an indexed format.
-      runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
-      runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
-      runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
-
-      PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
-          PCollectionList.of(ImmutableList.of(
-              perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
-
-      return Pipeline.applyTransform(outputs,
-                                     Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
-          .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
-                                        ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsMultimap";
-    }
-
-    static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
-        Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
-      // TODO: swap to use a variable length long coder which has values which compare
-      // the same as their byte representation compare lexicographically within the key coder
-      return IsmRecordCoder.of(
-          1, // We use only the key for hashing when producing value records
-          2, // Since the key is not present, we add the window to the hash when
-             // producing metadata records
-          ImmutableList.of(
-              MetadataKeyCoder.of(keyCoder),
-              windowCoder,
-              BigEndianLongCoder.of()),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
-   * {@code V1 -> V2}.
-   */
-  static class TransformedMap<K, V1, V2>
-      extends ForwardingMap<K, V2> {
-    private final Function<V1, V2> transform;
-    private final Map<K, V1> originalMap;
-    private final Map<K, V2> transformedMap;
-
-    private TransformedMap(Function<V1, V2> transform, Map<K, V1> originalMap) {
-      this.transform = transform;
-      this.originalMap = Collections.unmodifiableMap(originalMap);
-      this.transformedMap = Maps.transformValues(originalMap, transform);
-    }
-
-    @Override
-    protected Map<K, V2> delegate() {
-      return transformedMap;
-    }
-  }
-
-  /**
-   * A {@link Coder} for {@link TransformedMap}s.
-   */
-  static class TransformedMapCoder<K, V1, V2>
-      extends StandardCoder<TransformedMap<K, V1, V2>> {
-    private final Coder<Function<V1, V2>> transformCoder;
-    private final Coder<Map<K, V1>> originalMapCoder;
-
-    private TransformedMapCoder(
-        Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
-      this.transformCoder = transformCoder;
-      this.originalMapCoder = originalMapCoder;
-    }
-
-    public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
-        Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
-      return new TransformedMapCoder<>(transformCoder, originalMapCoder);
-    }
-
-    @JsonCreator
-    public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
-        @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-        List<Coder<?>> components) {
-      checkArgument(components.size() == 2,
-          "Expecting 2 components, got " + components.size());
-      @SuppressWarnings("unchecked")
-      Coder<Function<V1, V2>> transformCoder = (Coder<Function<V1, V2>>) components.get(0);
-      @SuppressWarnings("unchecked")
-      Coder<Map<K, V1>> originalMapCoder = (Coder<Map<K, V1>>) components.get(1);
-      return of(transformCoder, originalMapCoder);
-    }
-
-    @Override
-    public void encode(TransformedMap<K, V1, V2> value, OutputStr

<TRUNCATED>


[62/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
index e553dbb..4cdacec 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.runners.inprocess;
+
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
index 5c24672..479090f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
@@ -17,18 +17,18 @@
  */
 /**
  * Defines runners for executing Pipelines in different modes, including
- * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and
- * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
+ * {@link org.apache.beam.sdk.runners.DirectPipelineRunner} and
+ * {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
  *
- * <p>{@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline}
+ * <p>{@link org.apache.beam.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline}
  * locally, without contacting the Dataflow service.
- * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} submits a
+ * {@link org.apache.beam.sdk.runners.DataflowPipelineRunner} submits a
  * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine
  * instances. {@code DataflowPipelineRunner} returns
  * as soon as the {@code Pipeline} has been submitted. Use
- * {@link com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner} to have execution
+ * {@link org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner} to have execution
  * updates printed to the console.
  *
- * <p>The runner is specified as part {@link com.google.cloud.dataflow.sdk.options.PipelineOptions}.
+ * <p>The runner is specified as part {@link org.apache.beam.sdk.options.PipelineOptions}.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/IsmFormat.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/IsmFormat.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/IsmFormat.java
index dc822ea..315e3f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/IsmFormat.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/IsmFormat.java
@@ -15,27 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.worker;
+package org.apache.beam.sdk.runners.worker;
+
+import static org.apache.beam.sdk.util.Structs.addLong;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
 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 com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.RandomAccessData;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.RandomAccessData;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.base.MoreObjects.ToStringHelper;
 import com.google.common.base.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/package-info.java
index 00ecfdf..6133148 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/worker/package-info.java
@@ -20,6 +20,6 @@
  * execution of Pipeline code.
  */
 @ParametersAreNonnullByDefault
-package com.google.cloud.dataflow.sdk.runners.worker;
+package org.apache.beam.sdk.runners.worker;
 
 import javax.annotation.ParametersAreNonnullByDefault;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
index 8cd05d7..ceed8c0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -26,16 +26,17 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.Serializer;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.UnownedInputStream;
-import com.google.cloud.dataflow.sdk.util.UnownedOutputStream;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.Serializer;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.util.UnownedInputStream;
+import org.apache.beam.sdk.util.UnownedOutputStream;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
@@ -277,7 +278,7 @@ public class CoderProperties {
       + " If you truly intend to change the wire format for this Coder "
       + " then you must update getEncodingId() to a new value and add any supported"
       + " prior formats to getAllowedEncodings()."
-      + " See com.google.cloud.dataflow.sdk.coders.PrintBase64Encoding for how to generate"
+      + " See org.apache.beam.sdk.coders.PrintBase64Encoding for how to generate"
       + " new test data.";
 
   public static <T> void coderEncodesBase64(Coder<T> coder, T value, String base64Encoding)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 aebce42..f328c5b 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
@@ -15,38 +15,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.StreamingOptions;
+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.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.View;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+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.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/RunnableOnService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/RunnableOnService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/RunnableOnService.java
index ea14774..4c794dd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/RunnableOnService.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/RunnableOnService.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 /**
  * Category tag for validation tests which utilize {@link TestPipeline} for execution and

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
index 940e437..9132db7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import org.hamcrest.Matcher;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
index 689c387..21173d1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.values.KV;
+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.coders.ListCoder;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.values.KV;
+
 import com.google.common.base.MoreObjects;
 
 import org.hamcrest.BaseMatcher;
@@ -52,7 +53,7 @@ import javax.annotation.Nullable;
  * </ul>
  *
  * <p>For example, to match a group from
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}, which has type
+ * {@link org.apache.beam.sdk.transforms.GroupByKey}, which has type
  * {@code KV<K, Iterable<V>>} for some {@code K} and {@code V} and where the order of the iterable
  * is undefined, use a matcher like
  * {@code kv(equalTo("some key"), containsInAnyOrder(1, 2, 3))}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
index feff16a..e0b8890 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
@@ -25,11 +25,11 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.KV;
 
 import org.junit.Assert;
 import org.slf4j.Logger;
@@ -68,8 +68,8 @@ import java.util.concurrent.Future;
  *   use both.
  * </ul>
  * For example usages, see the unit tests of classes such as
- * {@link com.google.cloud.dataflow.sdk.io.AvroSource} or
- * {@link com.google.cloud.dataflow.sdk.io.XmlSource}.
+ * {@link org.apache.beam.sdk.io.AvroSource} or
+ * {@link org.apache.beam.sdk.io.XmlSource}.
  *
  * <p>Like {@link PAssert}, requires JUnit and Hamcrest to be present in the classpath.
  */
@@ -245,7 +245,7 @@ public class SourceTestUtils {
 
   /**
    * Expected outcome of
-   * {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader#splitAtFraction}.
+   * {@link org.apache.beam.sdk.io.BoundedSource.BoundedReader#splitAtFraction}.
    */
   public enum ExpectedSplitOutcome {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 17b1538..72c8cbc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
+package org.apache.beam.sdk.testing;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.util.TestCredential;
+
 import com.google.common.base.Optional;
 import com.google.common.base.Strings;
 import com.google.common.collect.Iterators;
@@ -53,7 +54,7 @@ import javax.annotation.Nullable;
  *   <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
  *   options. For example:
  *   <pre>{@code [
- *     "--runner=com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineRunner",
+ *     "--runner=org.apache.beam.sdk.testing.TestDataflowPipelineRunner",
  *     "--project=mygcpproject",
  *     "--stagingLocation=gs://mygcsbucket/path"
  *     ]}</pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java
index 2a30e86..2566a12 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.Matchers.greaterThan;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 
 import org.joda.time.Instant;
 import org.joda.time.ReadableInstant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
index d178b44..25b6bb2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/package-info.java
@@ -19,4 +19,4 @@
  * Defines utilities for unit testing Dataflow pipelines. The tests for the {@code PTransform}s and
  * examples included the Dataflow SDK provide examples of using these utilities.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
index 3cb6aab..4c98123 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
 
 /**
  * An {@code Aggregator<InputT>} enables monitoring of values of type {@code InputT},

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
index 8c09904..97961e9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java
index 8f050cc..fdbde74 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 6ed1c0f..2ed7a85 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
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
-import com.google.cloud.dataflow.sdk.util.WeightedValue;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn;
+import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
+import org.apache.beam.sdk.util.WeightedValue;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java
index d8e925f..4f9dfc4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.hash.Hashing;
 import com.google.common.hash.HashingOutputStream;
 import com.google.common.io.ByteStreams;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c6d2f58..28bbeed 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
@@ -15,45 +15,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.DelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.GlobalCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.RequiresContextInternal;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunner;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunners;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.DelegateCoder;
+import org.apache.beam.sdk.coders.IterableCoder;
+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;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.PerKeyCombineFnRunner;
+import org.apache.beam.sdk.util.PerKeyCombineFnRunners;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+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;
@@ -133,7 +135,7 @@ public class Combine {
    * <p>Each output element is in the window by which its corresponding input
    * was grouped, and has the timestamp of the end of that window.  The output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * as the input.
    *
    * <p>See {@link PerKey Combine.PerKey} for more information.
@@ -154,7 +156,7 @@ public class Combine {
    * <p>Each output element is in the window by which its corresponding input
    * was grouped, and has the timestamp of the end of that window.  The output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * as the input.
    *
    * <p>See {@link PerKey Combine.PerKey} for more information.
@@ -175,7 +177,7 @@ public class Combine {
    * <p>Each output element is in the window by which its corresponding input
    * was grouped, and has the timestamp of the end of that window.  The output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * as the input.
    *
    * <p>See {@link PerKey Combine.PerKey} for more information.
@@ -206,7 +208,7 @@ public class Combine {
    * <p>Each output element has the same timestamp and is in the same window
    * as its corresponding input element, and the output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * associated with it as the input.
    *
    * <p>See {@link GroupedValues Combine.GroupedValues} for more information.
@@ -232,7 +234,7 @@ public class Combine {
    * <p>Each output element has the same timestamp and is in the same window
    * as its corresponding input element, and the output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * associated with it as the input.
    *
    * <p>See {@link GroupedValues Combine.GroupedValues} for more information.
@@ -258,7 +260,7 @@ public class Combine {
    * <p>Each output element has the same timestamp and is in the same window
    * as its corresponding input element, and the output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * associated with it as the input.
    *
    * <p>See {@link GroupedValues Combine.GroupedValues} for more information.
@@ -1587,7 +1589,7 @@ public class Combine {
    * <p>Each output element is in the window by which its corresponding input
    * was grouped, and has the timestamp of the end of that window.  The output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * as the input.
    *
    * @param <K> the type of the keys of the input and output
@@ -2118,7 +2120,7 @@ public class Combine {
    * <p>Each output element has the same timestamp and is in the same window
    * as its corresponding input element, and the output
    * {@code PCollection} has the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+   * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
    * associated with it as the input.
    *
    * <p>See also {@link #globally}/{@link Globally Combine.Globally}, which

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java
index 0b097d6..a57d446 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
index 8120de3..d98bd13 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
@@ -15,25 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.GlobalCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.util.CombineFnUtil;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.util.CombineFnUtil;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
index db21b9e..77a7e53 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.values.PCollectionView;
 
 /**
  * This class contains combine functions that have access to {@code PipelineOptions} and side inputs
@@ -40,7 +40,7 @@ public class CombineWithContext {
   public abstract static class Context {
     /**
      * Returns the {@code PipelineOptions} specified with the
-     * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
+     * {@link org.apache.beam.sdk.runners.PipelineRunner}
      * invoking this {@code KeyedCombineFn}.
      */
     public abstract PipelineOptions getPipelineOptions();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java
index e3e7947..3bf264e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -82,7 +82,7 @@ public class Count {
    * <p>This transform compares two values of type {@code T} by first encoding each element using
    * the input {@code PCollection}'s {@code Coder}, then comparing the encoded bytes. Because of
    * this, the input coder must be deterministic.
-   * (See {@link com.google.cloud.dataflow.sdk.coders.Coder#verifyDeterministic()} for more detail).
+   * (See {@link org.apache.beam.sdk.coders.Coder#verifyDeterministic()} for more detail).
    * Performing the comparison in this manner admits efficient parallel evaluation.
    *
    * <p>By default, the {@code Coder} of the keys of the output {@code PCollection} is the same as

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 4bccdc1..27fb39d 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
@@ -15,22 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue.TimestampedValueCoder;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 74bb6ac..04f272d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -23,16 +23,17 @@ import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
+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.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.base.MoreObjects;
 
 import org.joda.time.Duration;
@@ -49,7 +50,7 @@ import java.util.UUID;
 /**
  * The argument to {@link ParDo} providing the code to use to process
  * elements of the input
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * {@link org.apache.beam.sdk.values.PCollection}.
  *
  * <p>See {@link ParDo} for more explanation, examples of use, and
  * discussion of constraints on {@code DoFn}s, including their
@@ -82,7 +83,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
 
     /**
      * Returns the {@code PipelineOptions} specified with the
-     * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
+     * {@link org.apache.beam.sdk.runners.PipelineRunner}
      * invoking this {@code DoFn}.  The {@code PipelineOptions} will
      * be the default running via {@link DoFnTester}.
      */
@@ -102,7 +103,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      *
      * <p>If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element. The output element
@@ -124,7 +125,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      *
      * <p>If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element except for the
@@ -149,7 +150,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      *
      * <p>If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element. The output element
@@ -173,7 +174,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      *
      * <p>If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element except for the
@@ -245,7 +246,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      * window of the main input element.
      *
      * <p>See
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn#getSideInputWindow}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn#getSideInputWindow}
      * for how this corresponding window is determined.
      *
      * @throws IllegalArgumentException if this is not a side input
@@ -256,7 +257,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
     /**
      * Returns the timestamp of the input element.
      *
-     * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window}
+     * <p>See {@link org.apache.beam.sdk.transforms.windowing.Window}
      * for more information.
      */
     public abstract Instant timestamp();
@@ -264,7 +265,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
     /**
      * Returns the window into which the input element has been assigned.
      *
-     * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window}
+     * <p>See {@link org.apache.beam.sdk.transforms.windowing.Window}
      * for more information.
      *
      * @throws UnsupportedOperationException if this {@link DoFn} does
@@ -278,7 +279,7 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      *
      * <p>Generally all data is in a single, uninteresting pane unless custom
      * triggering and/or late data has been explicitly requested.
-     * See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window}
+     * See {@link org.apache.beam.sdk.transforms.windowing.Window}
      * for more information.
      */
     public abstract PaneInfo pane();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 56af699..08c4391 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
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ExtraContextFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.FinishBundle;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ProcessElement;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.StartBundle;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ExtraContextFactory;
+import org.apache.beam.sdk.transforms.DoFnWithContext.FinishBundle;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ProcessElement;
+import org.apache.beam.sdk.transforms.DoFnWithContext.StartBundle;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+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.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Throwables;
@@ -119,7 +120,7 @@ public abstract class DoFnReflector {
    * Invoke the reflected {@link ProcessElement} method on the given instance.
    *
    * @param fn an instance of the {@link DoFnWithContext} to invoke {@link ProcessElement} on.
-   * @param c the {@link com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ProcessContext}
+   * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.ProcessContext}
    *     to pass to {@link ProcessElement}.
    */
   abstract <InputT, OutputT> void invokeProcessElement(
@@ -131,7 +132,7 @@ public abstract class DoFnReflector {
    * Invoke the reflected {@link StartBundle} method on the given instance.
    *
    * @param fn an instance of the {@link DoFnWithContext} to invoke {@link StartBundle} on.
-   * @param c the {@link com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.Context}
+   * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
    *     to pass to {@link StartBundle}.
    */
   <InputT, OutputT> void invokeStartBundle(
@@ -145,7 +146,7 @@ public abstract class DoFnReflector {
    * Invoke the reflected {@link FinishBundle} method on the given instance.
    *
    * @param fn an instance of the {@link DoFnWithContext} to invoke {@link FinishBundle} on.
-   * @param c the {@link com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.Context}
+   * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
    *     to pass to {@link FinishBundle}.
    */
   abstract <InputT, OutputT> void invokeFinishBundle(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 1b1bb71..1c60259 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -15,25 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext;
-import com.google.cloud.dataflow.sdk.util.DirectSideInputReader;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunnerBase;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.PTuple;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.DirectModeExecutionContext;
+import org.apache.beam.sdk.util.DirectSideInputReader;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunnerBase;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.PTuple;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
 import com.google.common.base.Function;
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterables;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java
index 34d5d2c..835730c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 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;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.DelegatingAggregator;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn.DelegatingAggregator;
+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.apache.beam.sdk.values.TypeDescriptor;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -47,7 +47,7 @@ import java.util.Map;
 /**
  * The argument to {@link ParDo} providing the code to use to process
  * elements of the input
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * {@link org.apache.beam.sdk.values.PCollection}.
  *
  * <p>See {@link ParDo} for more explanation, examples of use, and
  * discussion of constraints on {@code DoFnWithContext}s, including their
@@ -89,7 +89,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
 
     /**
      * Returns the {@code PipelineOptions} specified with the
-     * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
+     * {@link org.apache.beam.sdk.runners.PipelineRunner}
      * invoking this {@code DoFnWithContext}.  The {@code PipelineOptions} will
      * be the default running via {@link DoFnTester}.
      */
@@ -108,7 +108,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
      *
      * <p>If invoked from {@link StartBundle} or {@link FinishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element. The output element
@@ -130,7 +130,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
      *
      * <p>If invoked from {@link StartBundle} or {@link FinishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element except for the
@@ -155,7 +155,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
      *
      * <p>If invoked from {@link StartBundle} or {@link FinishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element. The output element
@@ -179,7 +179,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
      *
      * <p>If invoked from {@link StartBundle} or {@link FinishBundle},
      * this will attempt to use the
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+     * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
      * of the input {@code PCollection} to determine what windows the element
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element except for the
@@ -216,7 +216,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
     /**
      * Returns the timestamp of the input element.
      *
-     * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window}
+     * <p>See {@link org.apache.beam.sdk.transforms.windowing.Window}
      * for more information.
      */
     public abstract Instant timestamp();
@@ -227,7 +227,7 @@ public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
      *
      * <p>Generally all data is in a single, uninteresting pane unless custom
      * triggering and/or late data has been explicitly requested.
-     * See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window}
+     * See {@link org.apache.beam.sdk.transforms.windowing.Window}
      * for more information.
      */
     public abstract PaneInfo pane();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
index 30893b6..547254d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * {@code PTransform}s for filtering from a {@code PCollection} the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
index d5bbf2b..932643f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import java.lang.reflect.ParameterizedType;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
index 3db56d8..7c6fed3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.IterableLikeCoder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableLikeCoder;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
index 320d16d..42c1f78 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+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.PCollection.IsBounded;
 
 /**
  * {@code GroupByKey<K, V>} takes a {@code PCollection<KV<K, V>>},
@@ -80,7 +80,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
  * associated data together into one location.  It is also a key
  * determiner of the performance of a data-parallel pipeline.
  *
- * <p>See {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey}
+ * <p>See {@link org.apache.beam.sdk.transforms.join.CoGroupByKey}
  * for a way to group multiple input PCollections by a common key at once.
  *
  * <p>See {@link Combine.PerKey} for a common pattern of
@@ -91,7 +91,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
  * corresponding to the new, merged window will be created. The items in this pane
  * will be emitted when a trigger fires. By default this will be when the input
  * sources estimate there will be no more data for the window. See
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark}
+ * {@link org.apache.beam.sdk.transforms.windowing.AfterWatermark}
  * for details on the estimation.
  *
  * <p>The timestamp for each emitted pane is determined by the
@@ -100,7 +100,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
  * as the input.
  *
  * <p>If the input {@code PCollection} contains late data (see
- * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read.Bound#timestampLabel}
+ * {@link org.apache.beam.sdk.io.PubsubIO.Read.Bound#timestampLabel}
  * for an example of how this can occur) or the
  * {@link Window#triggering requested TriggerFn} can fire before
  * the watermark, then there may be multiple elements

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0a6787c..c66aa8d 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
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+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.PCollection;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java
index 2430336..a1e5e0a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * {@code Keys<K>} takes a {@code PCollection} of {@code KV<K, V>}s and
@@ -33,7 +33,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection;
  * <p>Each output element has the same timestamp and is in the same windows
  * as its corresponding input element, and the output {@code PCollection}
  * has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * associated with it as the input.
  *
  * <p>See also {@link Values}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java
index 5209f53..2dfc7c1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * {@code KvSwap<K, V>} takes a {@code PCollection<KV<K, V>>} and
@@ -35,7 +35,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection;
  * <p>Each output element has the same timestamp and is in the same windows
  * as its corresponding input element, and the output {@code PCollection}
  * has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * associated with it as the input.
  *
  * @param <K> the type of the keys in the input {@code PCollection}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
index c483e97..47c2f5d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
  * {@code PTransform}s for mapping a simple function over the elements of a {@link PCollection}.



[05/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
deleted file mode 100644
index 078827d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
+++ /dev/null
@@ -1,411 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * The evaluation context for a specific pipeline being executed by the
- * {@link InProcessPipelineRunner}. Contains state shared within the execution across all
- * transforms.
- *
- * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
- * {@link InProcessPipelineRunner} that can be used while evaluating a {@link PTransform}. This
- * consists of views into underlying state and watermark implementations, access to read and write
- * {@link PCollectionView PCollectionViews}, and constructing {@link CounterSet CounterSets} and
- * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when
- * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and
- * known to be empty).
- *
- * <p>{@link InProcessEvaluationContext} also handles results by committing finalizing bundles based
- * on the current global state and updating the global state appropriately. This includes updating
- * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that
- * can be executed.
- */
-class InProcessEvaluationContext {
-  /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
-  private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
-
-  /** The options that were used to create this {@link Pipeline}. */
-  private final InProcessPipelineOptions options;
-
-  private final BundleFactory bundleFactory;
-  /** The current processing time and event time watermarks and timers. */
-  private final InMemoryWatermarkManager watermarkManager;
-
-  /** Executes callbacks based on the progression of the watermark. */
-  private final WatermarkCallbackExecutor callbackExecutor;
-
-  /** The stateInternals of the world, by applied PTransform and key. */
-  private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals<?>>
-      applicationStateInternals;
-
-  private final InProcessSideInputContainer sideInputContainer;
-
-  private final CounterSet mergedCounters;
-
-  public static InProcessEvaluationContext create(
-      InProcessPipelineOptions options,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
-    return new InProcessEvaluationContext(
-        options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
-  }
-
-  private InProcessEvaluationContext(
-      InProcessPipelineOptions options,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
-    this.options = checkNotNull(options);
-    this.bundleFactory = checkNotNull(bundleFactory);
-    checkNotNull(rootTransforms);
-    checkNotNull(valueToConsumers);
-    checkNotNull(stepNames);
-    checkNotNull(views);
-    this.stepNames = stepNames;
-
-    this.watermarkManager =
-        InMemoryWatermarkManager.create(
-            NanosOffsetClock.create(), rootTransforms, valueToConsumers);
-    this.sideInputContainer = InProcessSideInputContainer.create(this, views);
-
-    this.applicationStateInternals = new ConcurrentHashMap<>();
-    this.mergedCounters = new CounterSet();
-
-    this.callbackExecutor = WatermarkCallbackExecutor.create();
-  }
-
-  /**
-   * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided
-   * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
-   *
-   * <p>The result is the output of running the transform contained in the
-   * {@link InProcessTransformResult} on the contents of the provided bundle.
-   *
-   * @param completedBundle the bundle that was processed to produce the result. Potentially
-   *                        {@code null} if the transform that produced the result is a root
-   *                        transform
-   * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
-   *                        or an empty iterable if no timers were delivered
-   * @param result the result of evaluating the input bundle
-   * @return the committed bundles contained within the handled {@code result}
-   */
-  public synchronized Iterable<? extends CommittedBundle<?>> handleResult(
-      @Nullable CommittedBundle<?> completedBundle,
-      Iterable<TimerData> completedTimers,
-      InProcessTransformResult result) {
-    Iterable<? extends CommittedBundle<?>> committedBundles =
-        commitBundles(result.getOutputBundles());
-    // Update watermarks and timers
-    watermarkManager.updateWatermarks(
-        completedBundle,
-        result.getTransform(),
-        result.getTimerUpdate().withCompletedTimers(completedTimers),
-        committedBundles,
-        result.getWatermarkHold());
-    fireAllAvailableCallbacks();
-    // Update counters
-    if (result.getCounters() != null) {
-      mergedCounters.merge(result.getCounters());
-    }
-    // Update state internals
-    CopyOnAccessInMemoryStateInternals<?> theirState = result.getState();
-    if (theirState != null) {
-      CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit();
-      StepAndKey stepAndKey =
-          StepAndKey.of(
-              result.getTransform(), completedBundle == null ? null : completedBundle.getKey());
-      if (!committedState.isEmpty()) {
-        applicationStateInternals.put(stepAndKey, committedState);
-      } else {
-        applicationStateInternals.remove(stepAndKey);
-      }
-    }
-    return committedBundles;
-  }
-
-  private Iterable<? extends CommittedBundle<?>> commitBundles(
-      Iterable<? extends UncommittedBundle<?>> bundles) {
-    ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
-    for (UncommittedBundle<?> inProgress : bundles) {
-      AppliedPTransform<?, ?, ?> producing =
-          inProgress.getPCollection().getProducingTransformInternal();
-      TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
-      CommittedBundle<?> committed =
-          inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
-      // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
-      // filter them out
-      if (!Iterables.isEmpty(committed.getElements())) {
-        completed.add(committed);
-      }
-    }
-    return completed.build();
-  }
-
-  private void fireAllAvailableCallbacks() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      fireAvailableCallbacks(transform);
-    }
-  }
-
-  private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) {
-    TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
-    callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark());
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} for use by a source.
-   */
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return bundleFactory.createRootBundle(output);
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} whose elements belong to the specified {@link
-   * PCollection}.
-   */
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return bundleFactory.createBundle(input, output);
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}.
-   */
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output) {
-    return bundleFactory.createKeyedBundle(input, key, output);
-  }
-
-  /**
-   * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided
-   * {@link PCollectionView}.
-   */
-  public <ElemT, ViewT> PCollectionViewWriter<ElemT, ViewT> createPCollectionViewWriter(
-      PCollection<Iterable<ElemT>> input, final PCollectionView<ViewT> output) {
-    return new PCollectionViewWriter<ElemT, ViewT>() {
-      @Override
-      public void add(Iterable<WindowedValue<ElemT>> values) {
-        sideInputContainer.write(output, values);
-      }
-    };
-  }
-
-  /**
-   * Schedule a callback to be executed after output would be produced for the given window
-   * if there had been input.
-   *
-   * <p>Output would be produced when the watermark for a {@link PValue} passes the point at
-   * which the trigger for the specified window (with the specified windowing strategy) must have
-   * fired from the perspective of that {@link PValue}, as specified by the value of
-   * {@link Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the
-   * {@link WindowingStrategy}. When the callback has fired, either values will have been produced
-   * for a key in that window, the window is empty, or all elements in the window are late. The
-   * callback will be executed regardless of whether values have been produced.
-   */
-  public void scheduleAfterOutputWouldBeProduced(
-      PValue value,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    AppliedPTransform<?, ?, ?> producing = getProducing(value);
-    callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
-
-    fireAvailableCallbacks(lookupProducing(value));
-  }
-
-  private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
-    if (value.getProducingTransformInternal() != null) {
-      return value.getProducingTransformInternal();
-    }
-    return lookupProducing(value);
-  }
-
-  private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
-        return transform;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Get the options used by this {@link Pipeline}.
-   */
-  public InProcessPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  /**
-   * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
-   */
-  public InProcessExecutionContext getExecutionContext(
-      AppliedPTransform<?, ?, ?> application, Object key) {
-    StepAndKey stepAndKey = StepAndKey.of(application, key);
-    return new InProcessExecutionContext(
-        options.getClock(),
-        key,
-        (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey),
-        watermarkManager.getWatermarks(application));
-  }
-
-  /**
-   * Get all of the steps used in this {@link Pipeline}.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
-    return stepNames.keySet();
-  }
-
-  /**
-   * Get the Step Name for the provided application.
-   */
-  public String getStepName(AppliedPTransform<?, ?, ?> application) {
-    return stepNames.get(application);
-  }
-
-  /**
-   * Returns a {@link SideInputReader} capable of reading the provided
-   * {@link PCollectionView PCollectionViews}.
-   * @param sideInputs the {@link PCollectionView PCollectionViews} the result should be able to
-   *                   read
-   * @return a {@link SideInputReader} that can read all of the provided
-   *         {@link PCollectionView PCollectionViews}
-   */
-  public SideInputReader createSideInputReader(final List<PCollectionView<?>> sideInputs) {
-    return sideInputContainer.createReaderForViews(sideInputs);
-  }
-
-  /**
-   * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent
-   * of all other {@link CounterSet CounterSets} created by this call.
-   *
-   * The {@link InProcessEvaluationContext} is responsible for unifying the counters present in
-   * all created {@link CounterSet CounterSets} when the transforms that call this method
-   * complete.
-   */
-  public CounterSet createCounterSet() {
-    return new CounterSet();
-  }
-
-  /**
-   * Returns all of the counters that have been merged into this context via calls to
-   * {@link CounterSet#merge(CounterSet)}.
-   */
-  public CounterSet getCounters() {
-    return mergedCounters;
-  }
-
-  /**
-   * Extracts all timers that have been fired and have not already been extracted.
-   *
-   * <p>This is a destructive operation. Timers will only appear in the result of this method once
-   * for each time they are set.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired =
-        watermarkManager.extractFiredTimers();
-    return fired;
-  }
-
-  /**
-   * Returns true if the step will not produce additional output.
-   *
-   * <p>If the provided transform produces only {@link IsBounded#BOUNDED}
-   * {@link PCollection PCollections}, returns true if the watermark is at
-   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE positive infinity}.
-   *
-   * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
-   * {@link PCollection PCollections}, returns the value of
-   * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
-   */
-  public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
-    // if the PTransform's watermark isn't at the max value, it isn't done
-    if (watermarkManager
-        .getWatermarks(transform)
-        .getOutputWatermark()
-        .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
-      return false;
-    }
-    // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
-    // the PTransform may produce additional output. It is not done.
-    for (PValue output : transform.getOutput().expand()) {
-      if (output instanceof PCollection) {
-        IsBounded bounded = ((PCollection<?>) output).isBounded();
-        if (bounded.equals(IsBounded.UNBOUNDED)
-            && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
-          return false;
-        }
-      }
-    }
-    // The PTransform's watermark was at positive infinity and all of its outputs are known to be
-    // done. It is done.
-    return true;
-  }
-
-  /**
-   * Returns true if all steps are done.
-   */
-  public boolean isDone() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      if (!isDone(transform)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
deleted file mode 100644
index adea2a6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
+++ /dev/null
@@ -1,108 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.BaseExecutionContext;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-/**
- * Execution Context for the {@link InProcessPipelineRunner}.
- *
- * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
- * for each thread that requires it.
- */
-class InProcessExecutionContext
-    extends BaseExecutionContext<InProcessExecutionContext.InProcessStepContext> {
-  private final Clock clock;
-  private final Object key;
-  private final CopyOnAccessInMemoryStateInternals<Object> existingState;
-  private final TransformWatermarks watermarks;
-
-  public InProcessExecutionContext(Clock clock, Object key,
-      CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
-    this.clock = clock;
-    this.key = key;
-    this.existingState = existingState;
-    this.watermarks = watermarks;
-  }
-
-  @Override
-  protected InProcessStepContext createStepContext(
-      String stepName, String transformName, StateSampler stateSampler) {
-    return new InProcessStepContext(this, stepName, transformName);
-  }
-
-  /**
-   * Step Context for the {@link InProcessPipelineRunner}.
-   */
-  public class InProcessStepContext
-      extends com.google.cloud.dataflow.sdk.util.BaseExecutionContext.StepContext {
-    private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
-    private InProcessTimerInternals timerInternals;
-
-    public InProcessStepContext(
-        ExecutionContext executionContext, String stepName, String transformName) {
-      super(executionContext, stepName, transformName);
-    }
-
-    @Override
-    public CopyOnAccessInMemoryStateInternals<Object> stateInternals() {
-      if (stateInternals == null) {
-        stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
-      }
-      return stateInternals;
-    }
-
-    @Override
-    public InProcessTimerInternals timerInternals() {
-      if (timerInternals == null) {
-        timerInternals =
-            InProcessTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
-      }
-      return timerInternals;
-    }
-
-    /**
-     * Commits the state of this step, and returns the committed state. If the step has not
-     * accessed any state, return null.
-     */
-    public CopyOnAccessInMemoryStateInternals<?> commitState() {
-      if (stateInternals != null) {
-        return stateInternals.commit();
-      }
-      return null;
-    }
-
-    /**
-     * Gets the timer update of the {@link TimerInternals} of this {@link InProcessStepContext},
-     * which is empty if the {@link TimerInternals} were never accessed.
-     */
-    public TimerUpdate getTimerUpdate() {
-      if (timerInternals == null) {
-        return TimerUpdate.empty();
-      }
-      return timerInternals.getTimerUpdate();
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java
deleted file mode 100644
index 2792fd3..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java
+++ /dev/null
@@ -1,48 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-import java.util.Collection;
-
-/**
- * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both
- * source and intermediate {@link PTransform PTransforms}.
- */
-interface InProcessExecutor {
-  /**
-   * Starts this executor. The provided collection is the collection of root transforms to
-   * initially schedule.
-   *
-   * @param rootTransforms
-   */
-  void start(Collection<AppliedPTransform<?, ?, ?>> rootTransforms);
-
-  /**
-   * Blocks until the job being executed enters a terminal state. A job is completed after all
-   * root {@link AppliedPTransform AppliedPTransforms} have completed, and all
-   * {@link CommittedBundle Bundles} have been consumed. Jobs may also terminate abnormally.
-   *
-   * @throws Throwable whenever an executor thread throws anything, transfers the throwable to the
-   *                   waiting thread and rethrows it
-   */
-  void awaitCompletion() throws Throwable;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java
deleted file mode 100644
index d44ea78..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java
+++ /dev/null
@@ -1,101 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.Hidden;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Options that can be used to configure the {@link InProcessPipelineRunner}.
- */
-public interface InProcessPipelineOptions extends PipelineOptions, ApplicationNameOptions {
-  /**
-   * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService}
-   * to execute {@link PTransform PTransforms}.
-   *
-   * <p>Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that
-   * it cannot enter a state in which it will not schedule additional pending work unless currently
-   * scheduled work completes, as this may cause the {@link Pipeline} to cease processing.
-   *
-   * <p>Defaults to a {@link CachedThreadPoolExecutorServiceFactory}, which produces instances of
-   * {@link Executors#newCachedThreadPool()}.
-   */
-  @JsonIgnore
-  @Required
-  @Hidden
-  @Default.InstanceFactory(CachedThreadPoolExecutorServiceFactory.class)
-  ExecutorServiceFactory getExecutorServiceFactory();
-
-  void setExecutorServiceFactory(ExecutorServiceFactory executorService);
-
-  /**
-   * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the
-   * system time when time values are required by the evaluator.
-   */
-  @Default.InstanceFactory(NanosOffsetClock.Factory.class)
-  @JsonIgnore
-  @Required
-  @Hidden
-  @Description(
-      "The processing time source used by the pipeline. When the current time is "
-          + "needed by the evaluator, the result of clock#now() is used.")
-  Clock getClock();
-
-  void setClock(Clock clock);
-
-  @Default.Boolean(false)
-  @Description(
-      "If the pipeline should shut down producers which have reached the maximum "
-          + "representable watermark. If this is set to true, a pipeline in which all PTransforms "
-          + "have reached the maximum watermark will be shut down, even if there are unbounded "
-          + "sources that could produce additional (late) data. By default, if the pipeline "
-          + "contains any unbounded PCollections, it will run until explicitly shut down.")
-  boolean isShutdownUnboundedProducersWithMaxWatermark();
-
-  void setShutdownUnboundedProducersWithMaxWatermark(boolean shutdown);
-
-  @Default.Boolean(true)
-  @Description(
-      "If the pipeline should block awaiting completion of the pipeline. If set to true, "
-          + "a call to Pipeline#run() will block until all PTransforms are complete. Otherwise, "
-          + "the Pipeline will execute asynchronously. If set to false, the completion of the "
-          + "pipeline can be awaited on by use of InProcessPipelineResult#awaitCompletion().")
-  boolean isBlockOnRun();
-
-  void setBlockOnRun(boolean b);
-
-  @Default.Boolean(true)
-  @Description(
-      "Controls whether the runner should ensure that all of the elements of every "
-          + "PCollection are not mutated. PTransforms are not permitted to mutate input elements "
-          + "at any point, or output elements after they are output.")
-  boolean isTestImmutability();
-
-  void setTestImmutability(boolean test);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
deleted file mode 100644
index 797a533..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
+++ /dev/null
@@ -1,366 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorPipelineExtractor;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessCreate.InProcessCreateOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
- * {@link PCollection PCollections}.
- */
-@Experimental
-public class InProcessPipelineRunner
-    extends PipelineRunner<InProcessPipelineRunner.InProcessPipelineResult> {
-  /**
-   * The default set of transform overrides to use in the {@link InProcessPipelineRunner}.
-   *
-   * <p>A transform override must have a single-argument constructor that takes an instance of the
-   * type of transform it is overriding.
-   */
-  @SuppressWarnings("rawtypes")
-  private static Map<Class<? extends PTransform>, PTransformOverrideFactory>
-      defaultTransformOverrides =
-          ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
-              .put(Create.Values.class, new InProcessCreateOverrideFactory())
-              .put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
-              .put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
-              .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
-              .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
-              .build();
-
-  /**
-   * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
-   * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is
-   * a part of at a later point. This is an uncommitted bundle and can have elements added to it.
-   *
-   * @param <T> the type of elements that can be added to this bundle
-   */
-  public static interface UncommittedBundle<T> {
-    /**
-     * Returns the PCollection that the elements of this {@link UncommittedBundle} belong to.
-     */
-    PCollection<T> getPCollection();
-
-    /**
-     * Outputs an element to this bundle.
-     *
-     * @param element the element to add to this bundle
-     * @return this bundle
-     */
-    UncommittedBundle<T> add(WindowedValue<T> element);
-
-    /**
-     * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
-     * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
-     * will throw an {@link IllegalStateException} if called after a call to commit.
-     * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
-     *                                   committed
-     */
-    CommittedBundle<T> commit(Instant synchronizedProcessingTime);
-  }
-
-  /**
-   * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
-   * eventually committed. Committed elements are executed by the {@link PTransform PTransforms}
-   * that consume the {@link PCollection} this bundle is
-   * a part of at a later point.
-   * @param <T> the type of elements contained within this bundle
-   */
-  public static interface CommittedBundle<T> {
-    /**
-     * Returns the PCollection that the elements of this bundle belong to.
-     */
-    PCollection<T> getPCollection();
-
-    /**
-     * Returns whether this bundle is keyed. A bundle that is part of a {@link PCollection} that
-     * occurs after a {@link GroupByKey} is keyed by the result of the last {@link GroupByKey}.
-     */
-    boolean isKeyed();
-
-    /**
-     * Returns the (possibly null) key that was output in the most recent {@link GroupByKey} in the
-     * execution of this bundle.
-     */
-    @Nullable
-    Object getKey();
-
-    /**
-     * Returns an {@link Iterable} containing all of the elements that have been added to this
-     * {@link CommittedBundle}.
-     */
-    Iterable<WindowedValue<T>> getElements();
-
-    /**
-     * Returns the processing time output watermark at the time the producing {@link PTransform}
-     * committed this bundle. Downstream synchronized processing time watermarks cannot progress
-     * past this point before consuming this bundle.
-     *
-     * <p>This value is no greater than the earliest incomplete processing time or synchronized
-     * processing time {@link TimerData timer} at the time this bundle was committed, including any
-     * timers that fired to produce this bundle.
-     */
-    Instant getSynchronizedProcessingOutputWatermark();
-  }
-
-  /**
-   * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to
-   * a storage mechanism that can be read from while constructing a {@link PCollectionView}.
-   * @param <ElemT> the type of elements the input {@link PCollection} contains.
-   * @param <ViewT> the type of the PCollectionView this writer writes to.
-   */
-  public static interface PCollectionViewWriter<ElemT, ViewT> {
-    void add(Iterable<WindowedValue<ElemT>> values);
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-  private final InProcessPipelineOptions options;
-
-  public static InProcessPipelineRunner fromOptions(PipelineOptions options) {
-    return new InProcessPipelineRunner(options.as(InProcessPipelineOptions.class));
-  }
-
-  private InProcessPipelineRunner(InProcessPipelineOptions options) {
-    this.options = options;
-  }
-
-  /**
-   * Returns the {@link PipelineOptions} used to create this {@link InProcessPipelineRunner}.
-   */
-  public InProcessPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass());
-    if (overrideFactory != null) {
-      PTransform<InputT, OutputT> customTransform = overrideFactory.override(transform);
-
-      return super.apply(customTransform, input);
-    }
-    // If there is no override, or we should not apply the override, apply the original transform
-    return super.apply(transform, input);
-  }
-
-  @Override
-  public InProcessPipelineResult run(Pipeline pipeline) {
-    ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
-    pipeline.traverseTopologically(consumerTrackingVisitor);
-    for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
-      unfinalized.finishSpecifying();
-    }
-    @SuppressWarnings("rawtypes")
-    KeyedPValueTrackingVisitor keyedPValueVisitor =
-        KeyedPValueTrackingVisitor.create(
-            ImmutableSet.<Class<? extends PTransform>>of(
-                GroupByKey.class, InProcessGroupByKeyOnly.class));
-    pipeline.traverseTopologically(keyedPValueVisitor);
-
-    InProcessEvaluationContext context =
-        InProcessEvaluationContext.create(
-            getPipelineOptions(),
-            createBundleFactory(getPipelineOptions()),
-            consumerTrackingVisitor.getRootTransforms(),
-            consumerTrackingVisitor.getValueToConsumers(),
-            consumerTrackingVisitor.getStepNames(),
-            consumerTrackingVisitor.getViews());
-
-    // independent executor service for each run
-    ExecutorService executorService =
-        context.getPipelineOptions().getExecutorServiceFactory().create();
-    InProcessExecutor executor =
-        ExecutorServiceParallelExecutor.create(
-            executorService,
-            consumerTrackingVisitor.getValueToConsumers(),
-            keyedPValueVisitor.getKeyedPValues(),
-            TransformEvaluatorRegistry.defaultRegistry(),
-            defaultModelEnforcements(options),
-            context);
-    executor.start(consumerTrackingVisitor.getRootTransforms());
-
-    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
-        new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
-    InProcessPipelineResult result =
-        new InProcessPipelineResult(executor, context, aggregatorSteps);
-    if (options.isBlockOnRun()) {
-      try {
-        result.awaitCompletion();
-      } catch (UserCodeException userException) {
-        throw new PipelineExecutionException(userException.getCause());
-      } catch (Throwable t) {
-        Throwables.propagate(t);
-      }
-    }
-    return result;
-  }
-
-  private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-      defaultModelEnforcements(InProcessPipelineOptions options) {
-    ImmutableMap.Builder<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-        enforcements = ImmutableMap.builder();
-    Collection<ModelEnforcementFactory> parDoEnforcements = createParDoEnforcements(options);
-    enforcements.put(ParDo.Bound.class, parDoEnforcements);
-    enforcements.put(ParDo.BoundMulti.class, parDoEnforcements);
-    return enforcements.build();
-  }
-
-  private Collection<ModelEnforcementFactory> createParDoEnforcements(
-      InProcessPipelineOptions options) {
-    ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
-    if (options.isTestImmutability()) {
-      enforcements.add(ImmutabilityEnforcementFactory.create());
-    }
-    return enforcements.build();
-  }
-
-  private BundleFactory createBundleFactory(InProcessPipelineOptions pipelineOptions) {
-    BundleFactory bundleFactory = InProcessBundleFactory.create();
-    if (pipelineOptions.isTestImmutability()) {
-      bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
-    }
-    return bundleFactory;
-  }
-
-  /**
-   * The result of running a {@link Pipeline} with the {@link InProcessPipelineRunner}.
-   *
-   * Throws {@link UnsupportedOperationException} for all methods.
-   */
-  public static class InProcessPipelineResult implements PipelineResult {
-    private final InProcessExecutor executor;
-    private final InProcessEvaluationContext evaluationContext;
-    private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
-    private State state;
-
-    private InProcessPipelineResult(
-        InProcessExecutor executor,
-        InProcessEvaluationContext evaluationContext,
-        Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
-      this.executor = executor;
-      this.evaluationContext = evaluationContext;
-      this.aggregatorSteps = aggregatorSteps;
-      // Only ever constructed after the executor has started.
-      this.state = State.RUNNING;
-    }
-
-    @Override
-    public State getState() {
-      return state;
-    }
-
-    @Override
-    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-        throws AggregatorRetrievalException {
-      CounterSet counters = evaluationContext.getCounters();
-      Collection<PTransform<?, ?>> steps = aggregatorSteps.get(aggregator);
-      Map<String, T> stepValues = new HashMap<>();
-      for (AppliedPTransform<?, ?, ?> transform : evaluationContext.getSteps()) {
-        if (steps.contains(transform.getTransform())) {
-          String stepName =
-              String.format(
-                  "user-%s-%s", evaluationContext.getStepName(transform), aggregator.getName());
-          Counter<T> counter = (Counter<T>) counters.getExistingCounter(stepName);
-          if (counter != null) {
-            stepValues.put(transform.getFullName(), counter.getAggregate());
-          }
-        }
-      }
-      return new MapAggregatorValues<>(stepValues);
-    }
-
-    /**
-     * Blocks until the {@link Pipeline} execution represented by this
-     * {@link InProcessPipelineResult} is complete, returning the terminal state.
-     *
-     * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
-     * exception. Future calls to {@link #getState()} will return
-     * {@link com.google.cloud.dataflow.sdk.PipelineResult.State#FAILED}.
-     *
-     * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
-     * {@link PCollection}, and the {@link PipelineRunner} was created with
-     * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
-     * this method will never return.
-     *
-     * See also {@link InProcessExecutor#awaitCompletion()}.
-     */
-    public State awaitCompletion() throws Throwable {
-      if (!state.isTerminal()) {
-        try {
-          executor.awaitCompletion();
-          state = State.DONE;
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw e;
-        } catch (Throwable t) {
-          state = State.FAILED;
-          throw t;
-        }
-      }
-      return state;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
deleted file mode 100644
index e031aa6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
+++ /dev/null
@@ -1,54 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessRegistrar {
-  private InProcessRegistrar() {}
-  /**
-   * Registers the {@link InProcessPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class InProcessRunner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(InProcessPipelineRunner.class);
-    }
-  }
-
-  /**
-   * Registers the {@link InProcessPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class InProcessOptions implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(InProcessPipelineOptions.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
deleted file mode 100644
index 70c2f20..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
+++ /dev/null
@@ -1,232 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Throwables;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.SettableFuture;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
-import javax.annotation.Nullable;
-
-/**
- * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for
- * constructing {@link SideInputReader SideInputReaders} which block until a side input is
- * available and writing to a {@link PCollectionView}.
- */
-class InProcessSideInputContainer {
-  private final InProcessEvaluationContext evaluationContext;
-  private final Collection<PCollectionView<?>> containedViews;
-  private final LoadingCache<PCollectionViewWindow<?>,
-      SettableFuture<Iterable<? extends WindowedValue<?>>>> viewByWindows;
-
-  /**
-   * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
-   * context.
-   */
-  public static InProcessSideInputContainer create(
-      InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
-    CacheLoader<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-        loader = new CacheLoader<PCollectionViewWindow<?>,
-            SettableFuture<Iterable<? extends WindowedValue<?>>>>() {
-          @Override
-          public SettableFuture<Iterable<? extends WindowedValue<?>>> load(
-              PCollectionViewWindow<?> view) {
-            return SettableFuture.create();
-          }
-        };
-    LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-        viewByWindows = CacheBuilder.newBuilder().build(loader);
-    return new InProcessSideInputContainer(context, containedViews, viewByWindows);
-  }
-
-  private InProcessSideInputContainer(InProcessEvaluationContext context,
-      Collection<PCollectionView<?>> containedViews,
-      LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-      viewByWindows) {
-    this.evaluationContext = context;
-    this.containedViews = ImmutableSet.copyOf(containedViews);
-    this.viewByWindows = viewByWindows;
-  }
-
-  /**
-   * Return a view of this {@link InProcessSideInputContainer} that contains only the views in
-   * the provided argument. The returned {@link InProcessSideInputContainer} is unmodifiable without
-   * casting, but will change as this {@link InProcessSideInputContainer} is modified.
-   */
-  public SideInputReader createReaderForViews(Collection<PCollectionView<?>> newContainedViews) {
-    if (!containedViews.containsAll(newContainedViews)) {
-      Set<PCollectionView<?>> currentlyContained = ImmutableSet.copyOf(containedViews);
-      Set<PCollectionView<?>> newRequested = ImmutableSet.copyOf(newContainedViews);
-      throw new IllegalArgumentException("Can't create a SideInputReader with unknown views "
-          + Sets.difference(newRequested, currentlyContained));
-    }
-    return new SideInputContainerSideInputReader(newContainedViews);
-  }
-
-  /**
-   * Write the provided values to the provided view.
-   *
-   * <p>The windowed values are first exploded, then for each window the pane is determined. For
-   * each window, if the pane is later than the current pane stored within this container, write
-   * all of the values to the container as the new values of the {@link PCollectionView}.
-   *
-   * <p>The provided iterable is expected to contain only a single window and pane.
-   */
-  public void write(PCollectionView<?> view, Iterable<? extends WindowedValue<?>> values) {
-    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow =
-        indexValuesByWindow(values);
-    for (Map.Entry<BoundedWindow, Collection<WindowedValue<?>>> windowValues :
-        valuesPerWindow.entrySet()) {
-      updatePCollectionViewWindowValues(view, windowValues.getKey(), windowValues.getValue());
-    }
-  }
-
-  /**
-   * Index the provided values by all {@link BoundedWindow windows} in which they appear.
-   */
-  private Map<BoundedWindow, Collection<WindowedValue<?>>> indexValuesByWindow(
-      Iterable<? extends WindowedValue<?>> values) {
-    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow = new HashMap<>();
-    for (WindowedValue<?> value : values) {
-      for (BoundedWindow window : value.getWindows()) {
-        Collection<WindowedValue<?>> windowValues = valuesPerWindow.get(window);
-        if (windowValues == null) {
-          windowValues = new ArrayList<>();
-          valuesPerWindow.put(window, windowValues);
-        }
-        windowValues.add(value);
-      }
-    }
-    return valuesPerWindow;
-  }
-
-  /**
-   * Set the value of the {@link PCollectionView} in the {@link BoundedWindow} to be based on the
-   * specified values, if the values are part of a later pane than currently exist within the
-   * {@link PCollectionViewWindow}.
-   */
-  private void updatePCollectionViewWindowValues(
-      PCollectionView<?> view, BoundedWindow window, Collection<WindowedValue<?>> windowValues) {
-    PCollectionViewWindow<?> windowedView = PCollectionViewWindow.of(view, window);
-    SettableFuture<Iterable<? extends WindowedValue<?>>> future = null;
-    try {
-      future = viewByWindows.get(windowedView);
-      if (future.isDone()) {
-        Iterator<? extends WindowedValue<?>> existingValues = future.get().iterator();
-        PaneInfo newPane = windowValues.iterator().next().getPane();
-        // The current value may have no elements, if no elements were produced for the window,
-        // but we are recieving late data.
-        if (!existingValues.hasNext()
-            || newPane.getIndex() > existingValues.next().getPane().getIndex()) {
-          viewByWindows.invalidate(windowedView);
-          viewByWindows.get(windowedView).set(windowValues);
-        }
-      } else {
-        future.set(windowValues);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      if (future != null && !future.isDone()) {
-        future.set(Collections.<WindowedValue<?>>emptyList());
-      }
-    } catch (ExecutionException e) {
-      Throwables.propagate(e.getCause());
-    }
-  }
-
-  private final class SideInputContainerSideInputReader implements SideInputReader {
-    private final Collection<PCollectionView<?>> readerViews;
-
-    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
-      this.readerViews = ImmutableSet.copyOf(readerViews);
-    }
-
-    @Override
-    @Nullable
-    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
-      checkArgument(
-          readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
-      PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
-      try {
-        final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
-            viewByWindows.get(windowedView);
-
-        WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
-        evaluationContext.scheduleAfterOutputWouldBeProduced(
-            view, window, windowingStrategy, new Runnable() {
-              @Override
-              public void run() {
-                // The requested window has closed without producing elements, so reflect that in
-                // the PCollectionView. If set has already been called, will do nothing.
-                future.set(Collections.<WindowedValue<?>>emptyList());
-          }
-
-          @Override
-          public String toString() {
-            return MoreObjects.toStringHelper("InProcessSideInputContainerEmptyCallback")
-                .add("view", view)
-                .add("window", window)
-                .toString();
-          }
-        });
-        // Safe covariant cast
-        @SuppressWarnings("unchecked")
-        Iterable<WindowedValue<?>> values = (Iterable<WindowedValue<?>>) future.get();
-        return view.fromIterableInternal(values);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        return null;
-      } catch (ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public <T> boolean contains(PCollectionView<T> view) {
-      return readerViews.contains(view);
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return readerViews.isEmpty();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java
deleted file mode 100644
index 9cd6069..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java
+++ /dev/null
@@ -1,85 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * An implementation of {@link TimerInternals} where all relevant data exists in memory.
- */
-public class InProcessTimerInternals implements TimerInternals {
-  private final Clock processingTimeClock;
-  private final TransformWatermarks watermarks;
-  private final TimerUpdateBuilder timerUpdateBuilder;
-
-  public static InProcessTimerInternals create(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    return new InProcessTimerInternals(clock, watermarks, timerUpdateBuilder);
-  }
-
-  private InProcessTimerInternals(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    this.processingTimeClock = clock;
-    this.watermarks = watermarks;
-    this.timerUpdateBuilder = timerUpdateBuilder;
-  }
-
-  @Override
-  public void setTimer(TimerData timerKey) {
-    timerUpdateBuilder.setTimer(timerKey);
-  }
-
-  @Override
-  public void deleteTimer(TimerData timerKey) {
-    timerUpdateBuilder.deletedTimer(timerKey);
-  }
-
-  public TimerUpdate getTimerUpdate() {
-    return timerUpdateBuilder.build();
-  }
-
-  @Override
-  public Instant currentProcessingTime() {
-    return processingTimeClock.now();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentSynchronizedProcessingTime() {
-    return watermarks.getSynchronizedProcessingInputTime();
-  }
-
-  @Override
-  public Instant currentInputWatermarkTime() {
-    return watermarks.getInputWatermark();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentOutputWatermarkTime() {
-    return watermarks.getOutputWatermark();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java
deleted file mode 100644
index 41d6672..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java
+++ /dev/null
@@ -1,77 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}.
- */
-public interface InProcessTransformResult {
-  /**
-   * Returns the {@link AppliedPTransform} that produced this result.
-   */
-  AppliedPTransform<?, ?, ?> getTransform();
-
-  /**
-   * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
-   * will be committed by the evaluation context as part of completing this result.
-   */
-  Iterable<? extends UncommittedBundle<?>> getOutputBundles();
-
-  /**
-   * Returns the {@link CounterSet} used by this {@link PTransform}, or null if this transform did
-   * not use a {@link CounterSet}.
-   */
-  @Nullable CounterSet getCounters();
-
-  /**
-   * Returns the Watermark Hold for the transform at the time this result was produced.
-   *
-   * If the transform does not set any watermark hold, returns
-   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
-   */
-  Instant getWatermarkHold();
-
-  /**
-   * Returns the State used by the transform.
-   *
-   * If this evaluation did not access state, this may return null.
-   */
-  @Nullable
-  CopyOnAccessInMemoryStateInternals<?> getState();
-
-  /**
-   * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
-   * evaluation was triggered due to the delivery of one or more timers, those timers must be added
-   * to the builder before it is complete.
-   *
-   * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
-   */
-  TimerUpdate getTimerUpdate();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
deleted file mode 100644
index 81a4801..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
+++ /dev/null
@@ -1,97 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
- * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that
- * produces keyed outputs is assumed to colocate output elements that share a key.
- *
- * <p>All {@link GroupByKey} transforms, or their runner-specific implementation primitive, produce
- * keyed output.
- */
-// TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
-// unkeyed
-class KeyedPValueTrackingVisitor implements PipelineVisitor {
-  @SuppressWarnings("rawtypes")
-  private final Set<Class<? extends PTransform>> producesKeyedOutputs;
-  private final Set<PValue> keyedValues;
-  private boolean finalized;
-
-  public static KeyedPValueTrackingVisitor create(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
-  }
-
-  private KeyedPValueTrackingVisitor(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    this.producesKeyedOutputs = producesKeyedOutputs;
-    this.keyedValues = new HashSet<>();
-  }
-
-  @Override
-  public void enterCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
-        KeyedPValueTrackingVisitor.class.getSimpleName(),
-        node);
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
-        KeyedPValueTrackingVisitor.class.getSimpleName(),
-        node);
-    if (node.isRootNode()) {
-      finalized = true;
-    } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
-      keyedValues.addAll(node.getExpandedOutputs());
-    }
-  }
-
-  @Override
-  public void visitTransform(TransformTreeNode node) {}
-
-  @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
-    if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
-      keyedValues.addAll(value.expand());
-    }
-  }
-
-  public Set<PValue> getKeyedPValues() {
-    checkState(
-        finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
-    return keyedValues;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java
deleted file mode 100644
index de2a77a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java
+++ /dev/null
@@ -1,63 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-/**
- * Enforcement tools that verify that executing code conforms to the model.
- *
- * <p>ModelEnforcement is performed on a per-element and per-bundle basis. The
- * {@link ModelEnforcement} is provided with the input bundle as part of
- * {@link ModelEnforcementFactory#forBundle(CommittedBundle, AppliedPTransform)}, each element
- * before and after that element is provided to an underlying {@link TransformEvaluator}, and the
- * output {@link InProcessTransformResult} and committed output bundles after the
- * {@link TransformEvaluator} has completed.
- *
- * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
- * of the input {@link PCollection} on construction, and then enforce per-element behavior
- * (such as the immutability of input elements). When the element is output or the bundle is
- * completed, the required conditions can be enforced across all elements.
- */
-public interface ModelEnforcement<T> {
-  /**
-   * Called before a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
-   * provided {@link WindowedValue}.
-   */
-  void beforeElement(WindowedValue<T> element);
-
-  /**
-   * Called after a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
-   * provided {@link WindowedValue}.
-   */
-  void afterElement(WindowedValue<T> element);
-
-  /**
-   * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
-   * called, producing the provided {@link InProcessTransformResult} and
-   * {@link CommittedBundle output bundles}.
-   */
-  void afterFinish(
-      CommittedBundle<T> input,
-      InProcessTransformResult result,
-      Iterable<? extends CommittedBundle<?>> outputs);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java
deleted file mode 100644
index 2bc01ce..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java
+++ /dev/null
@@ -1,30 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-
-/**
- * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input
- * {@link CommittedBundle bundle}. {@link ModelEnforcement} instances are created before the
- * {@link TransformEvaluator} is created.
- */
-public interface ModelEnforcementFactory {
-  <T> ModelEnforcement<T> forBundle(CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
deleted file mode 100644
index d033e4a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
+++ /dev/null
@@ -1,60 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time.
- */
-public class NanosOffsetClock implements Clock {
-  private final long baseMillis;
-  private final long nanosAtBaseMillis;
-
-  public static NanosOffsetClock create() {
-    return new NanosOffsetClock();
-  }
-
-  private NanosOffsetClock() {
-    baseMillis = System.currentTimeMillis();
-    nanosAtBaseMillis = System.nanoTime();
-  }
-
-  @Override
-  public Instant now() {
-    return new Instant(
-        baseMillis + (TimeUnit.MILLISECONDS.convert(
-            System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)));
-  }
-
-  /**
-   * Creates instances of {@link NanosOffsetClock}.
-   */
-  public static class Factory implements DefaultValueFactory<Clock> {
-    @Override
-    public Clock create(PipelineOptions options) {
-      return new NanosOffsetClock();
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java
deleted file mode 100644
index 77fa585..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java
+++ /dev/null
@@ -1,33 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-interface PTransformOverrideFactory {
-  /**
-   * Create a {@link PTransform} override for the provided {@link PTransform} if applicable.
-   * Otherwise, return the input {@link PTransform}.
-   *
-   * <p>The returned PTransform must be semantically equivalent to the input {@link PTransform}.
-   */
-  <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java
deleted file mode 100644
index 4b4d699..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-class ParDoInProcessEvaluator<T> implements TransformEvaluator<T> {
-  private final DoFnRunner<T, ?> fnRunner;
-  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
-  private final CounterSet counters;
-  private final Collection<UncommittedBundle<?>> outputBundles;
-  private final InProcessStepContext stepContext;
-
-  public ParDoInProcessEvaluator(
-      DoFnRunner<T, ?> fnRunner,
-      AppliedPTransform<PCollection<T>, ?, ?> transform,
-      CounterSet counters,
-      Collection<UncommittedBundle<?>> outputBundles,
-      InProcessStepContext stepContext) {
-    this.fnRunner = fnRunner;
-    this.transform = transform;
-    this.counters = counters;
-    this.outputBundles = outputBundles;
-    this.stepContext = stepContext;
-  }
-
-  @Override
-  public void processElement(WindowedValue<T> element) {
-    try {
-      fnRunner.processElement(element);
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-  }
-
-  @Override
-  public InProcessTransformResult finishBundle() {
-    try {
-      fnRunner.finishBundle();
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-    StepTransformResult.Builder resultBuilder;
-    CopyOnAccessInMemoryStateInternals<?> state = stepContext.commitState();
-    if (state != null) {
-      resultBuilder =
-          StepTransformResult.withHold(transform, state.getEarliestWatermarkHold())
-              .withState(state);
-    } else {
-      resultBuilder = StepTransformResult.withoutHold(transform);
-    }
-    return resultBuilder
-        .addOutput(outputBundles)
-        .withTimerUpdate(stepContext.getTimerUpdate())
-        .withCounters(counters)
-        .build();
-  }
-
-  static class BundleOutputManager implements OutputManager {
-    private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
-    private final Map<TupleTag<?>, List<?>> undeclaredOutputs;
-
-    public static BundleOutputManager create(Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
-      return new BundleOutputManager(outputBundles);
-    }
-
-    private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
-      this.bundles = bundles;
-      undeclaredOutputs = new HashMap<>();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-      @SuppressWarnings("rawtypes")
-      UncommittedBundle bundle = bundles.get(tag);
-      if (bundle == null) {
-        List undeclaredContents = undeclaredOutputs.get(tag);
-        if (undeclaredContents == null) {
-          undeclaredContents = new ArrayList<T>();
-          undeclaredOutputs.put(tag, undeclaredContents);
-        }
-        undeclaredContents.add(output);
-      } else {
-        bundle.add(output);
-      }
-    }
-  }
-}



[50/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSource.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSource.java b/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSource.java
deleted file mode 100644
index c8da9d2..0000000
--- a/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSource.java
+++ /dev/null
@@ -1,486 +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 com.google.cloud.dataflow.contrib.hadoop;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.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;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-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;
-import java.io.ObjectOutput;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.NoSuchElementException;
-import javax.annotation.Nullable;
-
-/**
- * A {@code BoundedSource} for reading files resident in a Hadoop filesystem using a
- * Hadoop file-based input format.
- *
- * <p>To read a {@link com.google.cloud.dataflow.sdk.values.PCollection} of
- * {@link com.google.cloud.dataflow.sdk.values.KV} key-value pairs from one or more
- * Hadoop files, use {@link HadoopFileSource#from} to specify the path(s) of the files to
- * read, the Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}, the
- * key class and the value class.
- *
- * <p>A {@code HadoopFileSource} can be read from using the
- * {@link com.google.cloud.dataflow.sdk.io.Read} transform. For example:
- *
- * <pre>
- * {@code
- * HadoopFileSource<K, V> source = HadoopFileSource.from(path, MyInputFormat.class,
- *   MyKey.class, MyValue.class);
- * PCollection<KV<MyKey, MyValue>> records = Read.from(mySource);
- * }
- * </pre>
- *
- * <p>The {@link HadoopFileSource#readFrom} method is a convenience method
- * that returns a read transform. For example:
- *
- * <pre>
- * {@code
- * PCollection<KV<MyKey, MyValue>> records = HadoopFileSource.readFrom(path,
- *   MyInputFormat.class, MyKey.class, MyValue.class);
- * }
- * </pre>
- *
- * Implementation note: Since Hadoop's {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
- * determines the input splits, this class extends {@link BoundedSource} rather than
- * {@link com.google.cloud.dataflow.sdk.io.OffsetBasedSource}, since the latter
- * dictates input splits.
-
- * @param <K> The type of keys to be read from the source.
- * @param <V> The type of values to be read from the source.
- */
-public class HadoopFileSource<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;
-
-  /**
-   * Creates a {@code Read} transform that will read from an {@code HadoopFileSource}
-   * with the given file name or pattern ("glob") using the given Hadoop
-   * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat},
-   * with key-value types specified by the given key class and value class.
-   */
-  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) {
-    return Read.from(from(filepattern, formatClass, keyClass, valueClass));
-  }
-
-  /**
-   * Creates a {@code HadoopFileSource} that reads from the given file name or pattern ("glob")
-   * using the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat},
-   * with key-value types specified by the given key class and value class.
-   */
-  public static <K, V, T extends FileInputFormat<K, V>> HadoopFileSource<K, V> from(
-      String filepattern, Class<T> formatClass, Class<K> keyClass, Class<V> valueClass) {
-    @SuppressWarnings("unchecked")
-    HadoopFileSource<K, V> source = (HadoopFileSource<K, V>)
-        new HadoopFileSource(filepattern, formatClass, keyClass, valueClass);
-    return source;
-  }
-
-  /**
-   * Create a {@code HadoopFileSource} based on a file or a file pattern specification.
-   */
-  private HadoopFileSource(String filepattern,
-      Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
-      Class<V> valueClass) {
-    this(filepattern, formatClass, keyClass, valueClass, null);
-  }
-
-  /**
-   * Create a {@code HadoopFileSource} based on a single Hadoop input split, which won't be
-   * split up further.
-   */
-  private HadoopFileSource(String filepattern,
-      Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
-      Class<V> valueClass, SerializableSplit serializableSplit) {
-    this.filepattern = filepattern;
-    this.formatClass = formatClass;
-    this.keyClass = keyClass;
-    this.valueClass = valueClass;
-    this.serializableSplit = serializableSplit;
-  }
-
-  public String getFilepattern() {
-    return filepattern;
-  }
-
-  public Class<? extends FileInputFormat<?, ?>> getFormatClass() {
-    return formatClass;
-  }
-
-  public Class<K> getKeyClass() {
-    return keyClass;
-  }
-
-  public Class<V> getValueClass() {
-    return valueClass;
-  }
-
-  @Override
-  public void validate() {
-    Preconditions.checkNotNull(filepattern,
-        "need to set the filepattern of a HadoopFileSource");
-    Preconditions.checkNotNull(formatClass,
-        "need to set the format class of a HadoopFileSource");
-    Preconditions.checkNotNull(keyClass,
-        "need to set the key class of a HadoopFileSource");
-    Preconditions.checkNotNull(valueClass,
-        "need to set the value class of a HadoopFileSource");
-  }
-
-  @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 HadoopFileSource<K, V>(filepattern, formatClass, keyClass,
-              valueClass, new SerializableSplit(inputSplit));
-        }
-      });
-    } else {
-      return ImmutableList.of(this);
-    }
-  }
-
-  private FileInputFormat<?, ?> createFormat(Job job) throws IOException, IllegalAccessException,
-      InstantiationException {
-    Path path = new Path(filepattern);
-    FileInputFormat.addInputPath(job, path);
-    return formatClass.newInstance();
-  }
-
-  private List<InputSplit> computeSplits(long desiredBundleSizeBytes) throws IOException,
-      IllegalAccessException, InstantiationException {
-    Job job = Job.getInstance();
-    FileInputFormat.setMinInputSplitSize(job, desiredBundleSizeBytes);
-    FileInputFormat.setMaxInputSplitSize(job, desiredBundleSizeBytes);
-    return createFormat(job).getSplits(job);
-  }
-
-  @Override
-  public BoundedReader<KV<K, V>> createReader(PipelineOptions options) throws IOException {
-    this.validate();
-
-    if (serializableSplit == null) {
-      return new HadoopFileReader<>(this, filepattern, formatClass);
-    } else {
-      return new HadoopFileReader<>(this, filepattern, formatClass,
-          serializableSplit.getSplit());
-    }
-  }
-
-  @Override
-  public Coder<KV<K, V>> getDefaultOutputCoder() {
-    return KvCoder.of(getDefaultCoder(keyClass), getDefaultCoder(valueClass));
-  }
-
-  @SuppressWarnings("unchecked")
-  private <T> Coder<T> getDefaultCoder(Class<T> c) {
-    if (Writable.class.isAssignableFrom(c)) {
-      Class<? extends Writable> writableClass = (Class<? extends Writable>) c;
-      return (Coder<T>) WritableCoder.of(writableClass);
-    } else if (Void.class.equals(c)) {
-      return (Coder<T>) VoidCoder.of();
-    }
-    // TODO: how to use registered coders here?
-    throw new IllegalStateException("Cannot find coder for " + c);
-  }
-
-  // BoundedSource
-
-  @Override
-  public long getEstimatedSizeBytes(PipelineOptions options) {
-    long size = 0;
-    try {
-      Job job = Job.getInstance(); // new instance
-      for (FileStatus st : listStatus(createFormat(job), job)) {
-        size += st.getLen();
-      }
-    } catch (IOException | NoSuchMethodException | InvocationTargetException
-        | IllegalAccessException | InstantiationException e) {
-      // ignore, and return 0
-    }
-    return size;
-  }
-
-  private <K, V> List<FileStatus> listStatus(FileInputFormat<K, V> format,
-      JobContext jobContext) throws NoSuchMethodException, InvocationTargetException,
-      IllegalAccessException {
-    // FileInputFormat#listStatus is protected, so call using reflection
-    Method listStatus = FileInputFormat.class.getDeclaredMethod("listStatus", JobContext.class);
-    listStatus.setAccessible(true);
-    @SuppressWarnings("unchecked")
-    List<FileStatus> stat = (List<FileStatus>) listStatus.invoke(format, jobContext);
-    return stat;
-  }
-
-  @Override
-  public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-    return false;
-  }
-
-  static class HadoopFileReader<K, V> extends BoundedSource.BoundedReader<KV<K, V>> {
-
-    private final BoundedSource<KV<K, V>> source;
-    private final String filepattern;
-    private final Class formatClass;
-
-    private FileInputFormat<?, ?> format;
-    private TaskAttemptContext attemptContext;
-    private List<InputSplit> splits;
-    private ListIterator<InputSplit> splitsIterator;
-    private Configuration conf;
-    private RecordReader<K, V> currentReader;
-    private KV<K, V> currentPair;
-
-    /**
-     * Create a {@code HadoopFileReader} based on a file or a file pattern specification.
-     */
-    public HadoopFileReader(BoundedSource<KV<K, V>> source, String filepattern,
-        Class<? extends FileInputFormat<?, ?>> formatClass) {
-      this(source, filepattern, formatClass, null);
-    }
-
-    /**
-     * Create a {@code HadoopFileReader} based on a single Hadoop input split.
-     */
-    public HadoopFileReader(BoundedSource<KV<K, V>> source, String filepattern,
-        Class<? extends FileInputFormat<?, ?>> formatClass, InputSplit split) {
-      this.source = source;
-      this.filepattern = filepattern;
-      this.formatClass = formatClass;
-      if (split != null) {
-        this.splits = ImmutableList.of(split);
-        this.splitsIterator = splits.listIterator();
-      }
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      Job job = Job.getInstance(); // new instance
-      Path path = new Path(filepattern);
-      FileInputFormat.addInputPath(job, path);
-
-      try {
-        @SuppressWarnings("unchecked")
-        FileInputFormat<K, V> f = (FileInputFormat<K, V>) formatClass.newInstance();
-        this.format = f;
-      } catch (InstantiationException | IllegalAccessException e) {
-        throw new IOException("Cannot instantiate file input format " + formatClass, e);
-      }
-      this.attemptContext = new TaskAttemptContextImpl(job.getConfiguration(),
-          new TaskAttemptID());
-
-      if (splitsIterator == null) {
-        this.splits = format.getSplits(job);
-        this.splitsIterator = splits.listIterator();
-      }
-      this.conf = job.getConfiguration();
-      return advance();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      try {
-        if (currentReader != null && currentReader.nextKeyValue()) {
-          currentPair = nextPair();
-          return true;
-        } else {
-          while (splitsIterator.hasNext()) {
-            // advance the reader and see if it has records
-            InputSplit nextSplit = splitsIterator.next();
-            @SuppressWarnings("unchecked")
-            RecordReader<K, V> reader =
-                (RecordReader<K, V>) format.createRecordReader(nextSplit, attemptContext);
-            if (currentReader != null) {
-              currentReader.close();
-            }
-            currentReader = reader;
-            currentReader.initialize(nextSplit, attemptContext);
-            if (currentReader.nextKeyValue()) {
-              currentPair = nextPair();
-              return true;
-            }
-            currentReader.close();
-            currentReader = null;
-          }
-          // either no next split or all readers were empty
-          currentPair = null;
-          return false;
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException(e);
-      }
-    }
-
-    @SuppressWarnings("unchecked")
-    private 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
-      if (key instanceof Writable) {
-        key = (K) WritableUtils.clone((Writable) key, conf);
-      }
-      if (value instanceof Writable) {
-        value = (V) WritableUtils.clone((Writable) value, conf);
-      }
-      return KV.of(key, value);
-    }
-
-    @Override
-    public KV<K, V> getCurrent() throws NoSuchElementException {
-      if (currentPair == null) {
-        throw new NoSuchElementException();
-      }
-      return currentPair;
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (currentReader != null) {
-        currentReader.close();
-        currentReader = null;
-      }
-      currentPair = null;
-    }
-
-    @Override
-    public BoundedSource<KV<K, V>> getCurrentSource() {
-      return source;
-    }
-
-    // BoundedReader
-
-    @Override
-    public Double getFractionConsumed() {
-      if (currentReader == null) {
-        return 0.0;
-      }
-      if (splits.isEmpty()) {
-        return 1.0;
-      }
-      int index = splitsIterator.previousIndex();
-      int numReaders = splits.size();
-      if (index == numReaders) {
-        return 1.0;
-      }
-      double before = 1.0 * index / numReaders;
-      double after = 1.0 * (index + 1) / numReaders;
-      Double fractionOfCurrentReader = getProgress();
-      if (fractionOfCurrentReader == null) {
-        return before;
-      }
-      return before + fractionOfCurrentReader * (after - before);
-    }
-
-    private Double getProgress() {
-      try {
-        return (double) currentReader.getProgress();
-      } catch (IOException | InterruptedException e) {
-        return null;
-      }
-    }
-
-    @Override
-    public BoundedSource<KV<K, V>> splitAtFraction(double fraction) {
-      // Not yet supported. To implement this, the sizes of the splits should be used to
-      // calculate the remaining splits that constitute the given fraction, then a
-      // new source backed by those splits should be returned.
-      return null;
-    }
-  }
-
-  /**
-   * A wrapper to allow Hadoop {@link org.apache.hadoop.mapreduce.InputSplit}s to be
-   * serialized using Java's standard serialization mechanisms. Note that the InputSplit
-   * has to be Writable (which most are).
-   */
-  public static class SerializableSplit implements Externalizable {
-    private static final long serialVersionUID = 0L;
-
-    private InputSplit split;
-
-    public SerializableSplit() {
-    }
-
-    public SerializableSplit(InputSplit split) {
-      Preconditions.checkArgument(split instanceof Writable, "Split is not writable: "
-          + split);
-      this.split = split;
-    }
-
-    public InputSplit getSplit() {
-      return split;
-    }
-
-    @Override
-    public void writeExternal(ObjectOutput out) throws IOException {
-      out.writeUTF(split.getClass().getCanonicalName());
-      ((Writable) split).write(out);
-    }
-
-    @Override
-    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-      String className = in.readUTF();
-      try {
-        split = (InputSplit) Class.forName(className).newInstance();
-        ((Writable) split).readFields(in);
-      } catch (InstantiationException | IllegalAccessException e) {
-        throw new IOException(e);
-      }
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoder.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoder.java b/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoder.java
deleted file mode 100644
index 572c24b..0000000
--- a/contrib/hadoop/src/main/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoder.java
+++ /dev/null
@@ -1,111 +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 com.google.cloud.dataflow.contrib.hadoop;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.hadoop.io.Writable;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-
-/**
- * A {@code WritableCoder} is a {@link com.google.cloud.dataflow.sdk.coders.Coder} for a
- * Java class that implements {@link org.apache.hadoop.io.Writable}.
- *
- * <p> To use, specify the coder type on a PCollection:
- * <pre>
- * {@code
- *   PCollection<MyRecord> records =
- *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
- * }
- * </pre>
- *
- * @param <T> the type of elements handled by this coder
- */
-public class WritableCoder<T extends Writable> extends StandardCoder<T> {
-  private static final long serialVersionUID = 0L;
-
-  /**
-   * Returns a {@code WritableCoder} instance for the provided element class.
-   * @param <T> the element type
-   */
-  public static <T extends Writable> WritableCoder<T> of(Class<T> clazz) {
-    return new WritableCoder<>(clazz);
-  }
-
-  @JsonCreator
-  @SuppressWarnings("unchecked")
-  public static WritableCoder<?> of(@JsonProperty("type") String classType)
-      throws ClassNotFoundException {
-    Class<?> clazz = Class.forName(classType);
-    if (!Writable.class.isAssignableFrom(clazz)) {
-      throw new ClassNotFoundException(
-          "Class " + classType + " does not implement Writable");
-    }
-    return of((Class<? extends Writable>) clazz);
-  }
-
-  private final Class<T> type;
-
-  public WritableCoder(Class<T> type) {
-    this.type = type;
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context) throws IOException {
-    value.write(new DataOutputStream(outStream));
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws IOException {
-    try {
-      T t = type.newInstance();
-      t.readFields(new DataInputStream(inStream));
-      return t;
-    } catch (InstantiationException | IllegalAccessException e) {
-      throw new CoderException("unable to deserialize record", e);
-    }
-  }
-
-  @Override
-  public List<Coder<?>> getCoderArguments() {
-    return null;
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    result.put("type", type.getName());
-    return result;
-  }
-
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Hadoop Writable may be non-deterministic.");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
new file mode 100644
index 0000000..c8da9d2
--- /dev/null
+++ b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
@@ -0,0 +1,486 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.hadoop;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.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;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+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;
+import java.io.ObjectOutput;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+
+/**
+ * A {@code BoundedSource} for reading files resident in a Hadoop filesystem using a
+ * Hadoop file-based input format.
+ *
+ * <p>To read a {@link com.google.cloud.dataflow.sdk.values.PCollection} of
+ * {@link com.google.cloud.dataflow.sdk.values.KV} key-value pairs from one or more
+ * Hadoop files, use {@link HadoopFileSource#from} to specify the path(s) of the files to
+ * read, the Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}, the
+ * key class and the value class.
+ *
+ * <p>A {@code HadoopFileSource} can be read from using the
+ * {@link com.google.cloud.dataflow.sdk.io.Read} transform. For example:
+ *
+ * <pre>
+ * {@code
+ * HadoopFileSource<K, V> source = HadoopFileSource.from(path, MyInputFormat.class,
+ *   MyKey.class, MyValue.class);
+ * PCollection<KV<MyKey, MyValue>> records = Read.from(mySource);
+ * }
+ * </pre>
+ *
+ * <p>The {@link HadoopFileSource#readFrom} method is a convenience method
+ * that returns a read transform. For example:
+ *
+ * <pre>
+ * {@code
+ * PCollection<KV<MyKey, MyValue>> records = HadoopFileSource.readFrom(path,
+ *   MyInputFormat.class, MyKey.class, MyValue.class);
+ * }
+ * </pre>
+ *
+ * Implementation note: Since Hadoop's {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
+ * determines the input splits, this class extends {@link BoundedSource} rather than
+ * {@link com.google.cloud.dataflow.sdk.io.OffsetBasedSource}, since the latter
+ * dictates input splits.
+
+ * @param <K> The type of keys to be read from the source.
+ * @param <V> The type of values to be read from the source.
+ */
+public class HadoopFileSource<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;
+
+  /**
+   * Creates a {@code Read} transform that will read from an {@code HadoopFileSource}
+   * with the given file name or pattern ("glob") using the given Hadoop
+   * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat},
+   * with key-value types specified by the given key class and value class.
+   */
+  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) {
+    return Read.from(from(filepattern, formatClass, keyClass, valueClass));
+  }
+
+  /**
+   * Creates a {@code HadoopFileSource} that reads from the given file name or pattern ("glob")
+   * using the given Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat},
+   * with key-value types specified by the given key class and value class.
+   */
+  public static <K, V, T extends FileInputFormat<K, V>> HadoopFileSource<K, V> from(
+      String filepattern, Class<T> formatClass, Class<K> keyClass, Class<V> valueClass) {
+    @SuppressWarnings("unchecked")
+    HadoopFileSource<K, V> source = (HadoopFileSource<K, V>)
+        new HadoopFileSource(filepattern, formatClass, keyClass, valueClass);
+    return source;
+  }
+
+  /**
+   * Create a {@code HadoopFileSource} based on a file or a file pattern specification.
+   */
+  private HadoopFileSource(String filepattern,
+      Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
+      Class<V> valueClass) {
+    this(filepattern, formatClass, keyClass, valueClass, null);
+  }
+
+  /**
+   * Create a {@code HadoopFileSource} based on a single Hadoop input split, which won't be
+   * split up further.
+   */
+  private HadoopFileSource(String filepattern,
+      Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
+      Class<V> valueClass, SerializableSplit serializableSplit) {
+    this.filepattern = filepattern;
+    this.formatClass = formatClass;
+    this.keyClass = keyClass;
+    this.valueClass = valueClass;
+    this.serializableSplit = serializableSplit;
+  }
+
+  public String getFilepattern() {
+    return filepattern;
+  }
+
+  public Class<? extends FileInputFormat<?, ?>> getFormatClass() {
+    return formatClass;
+  }
+
+  public Class<K> getKeyClass() {
+    return keyClass;
+  }
+
+  public Class<V> getValueClass() {
+    return valueClass;
+  }
+
+  @Override
+  public void validate() {
+    Preconditions.checkNotNull(filepattern,
+        "need to set the filepattern of a HadoopFileSource");
+    Preconditions.checkNotNull(formatClass,
+        "need to set the format class of a HadoopFileSource");
+    Preconditions.checkNotNull(keyClass,
+        "need to set the key class of a HadoopFileSource");
+    Preconditions.checkNotNull(valueClass,
+        "need to set the value class of a HadoopFileSource");
+  }
+
+  @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 HadoopFileSource<K, V>(filepattern, formatClass, keyClass,
+              valueClass, new SerializableSplit(inputSplit));
+        }
+      });
+    } else {
+      return ImmutableList.of(this);
+    }
+  }
+
+  private FileInputFormat<?, ?> createFormat(Job job) throws IOException, IllegalAccessException,
+      InstantiationException {
+    Path path = new Path(filepattern);
+    FileInputFormat.addInputPath(job, path);
+    return formatClass.newInstance();
+  }
+
+  private List<InputSplit> computeSplits(long desiredBundleSizeBytes) throws IOException,
+      IllegalAccessException, InstantiationException {
+    Job job = Job.getInstance();
+    FileInputFormat.setMinInputSplitSize(job, desiredBundleSizeBytes);
+    FileInputFormat.setMaxInputSplitSize(job, desiredBundleSizeBytes);
+    return createFormat(job).getSplits(job);
+  }
+
+  @Override
+  public BoundedReader<KV<K, V>> createReader(PipelineOptions options) throws IOException {
+    this.validate();
+
+    if (serializableSplit == null) {
+      return new HadoopFileReader<>(this, filepattern, formatClass);
+    } else {
+      return new HadoopFileReader<>(this, filepattern, formatClass,
+          serializableSplit.getSplit());
+    }
+  }
+
+  @Override
+  public Coder<KV<K, V>> getDefaultOutputCoder() {
+    return KvCoder.of(getDefaultCoder(keyClass), getDefaultCoder(valueClass));
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T> Coder<T> getDefaultCoder(Class<T> c) {
+    if (Writable.class.isAssignableFrom(c)) {
+      Class<? extends Writable> writableClass = (Class<? extends Writable>) c;
+      return (Coder<T>) WritableCoder.of(writableClass);
+    } else if (Void.class.equals(c)) {
+      return (Coder<T>) VoidCoder.of();
+    }
+    // TODO: how to use registered coders here?
+    throw new IllegalStateException("Cannot find coder for " + c);
+  }
+
+  // BoundedSource
+
+  @Override
+  public long getEstimatedSizeBytes(PipelineOptions options) {
+    long size = 0;
+    try {
+      Job job = Job.getInstance(); // new instance
+      for (FileStatus st : listStatus(createFormat(job), job)) {
+        size += st.getLen();
+      }
+    } catch (IOException | NoSuchMethodException | InvocationTargetException
+        | IllegalAccessException | InstantiationException e) {
+      // ignore, and return 0
+    }
+    return size;
+  }
+
+  private <K, V> List<FileStatus> listStatus(FileInputFormat<K, V> format,
+      JobContext jobContext) throws NoSuchMethodException, InvocationTargetException,
+      IllegalAccessException {
+    // FileInputFormat#listStatus is protected, so call using reflection
+    Method listStatus = FileInputFormat.class.getDeclaredMethod("listStatus", JobContext.class);
+    listStatus.setAccessible(true);
+    @SuppressWarnings("unchecked")
+    List<FileStatus> stat = (List<FileStatus>) listStatus.invoke(format, jobContext);
+    return stat;
+  }
+
+  @Override
+  public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+    return false;
+  }
+
+  static class HadoopFileReader<K, V> extends BoundedSource.BoundedReader<KV<K, V>> {
+
+    private final BoundedSource<KV<K, V>> source;
+    private final String filepattern;
+    private final Class formatClass;
+
+    private FileInputFormat<?, ?> format;
+    private TaskAttemptContext attemptContext;
+    private List<InputSplit> splits;
+    private ListIterator<InputSplit> splitsIterator;
+    private Configuration conf;
+    private RecordReader<K, V> currentReader;
+    private KV<K, V> currentPair;
+
+    /**
+     * Create a {@code HadoopFileReader} based on a file or a file pattern specification.
+     */
+    public HadoopFileReader(BoundedSource<KV<K, V>> source, String filepattern,
+        Class<? extends FileInputFormat<?, ?>> formatClass) {
+      this(source, filepattern, formatClass, null);
+    }
+
+    /**
+     * Create a {@code HadoopFileReader} based on a single Hadoop input split.
+     */
+    public HadoopFileReader(BoundedSource<KV<K, V>> source, String filepattern,
+        Class<? extends FileInputFormat<?, ?>> formatClass, InputSplit split) {
+      this.source = source;
+      this.filepattern = filepattern;
+      this.formatClass = formatClass;
+      if (split != null) {
+        this.splits = ImmutableList.of(split);
+        this.splitsIterator = splits.listIterator();
+      }
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      Job job = Job.getInstance(); // new instance
+      Path path = new Path(filepattern);
+      FileInputFormat.addInputPath(job, path);
+
+      try {
+        @SuppressWarnings("unchecked")
+        FileInputFormat<K, V> f = (FileInputFormat<K, V>) formatClass.newInstance();
+        this.format = f;
+      } catch (InstantiationException | IllegalAccessException e) {
+        throw new IOException("Cannot instantiate file input format " + formatClass, e);
+      }
+      this.attemptContext = new TaskAttemptContextImpl(job.getConfiguration(),
+          new TaskAttemptID());
+
+      if (splitsIterator == null) {
+        this.splits = format.getSplits(job);
+        this.splitsIterator = splits.listIterator();
+      }
+      this.conf = job.getConfiguration();
+      return advance();
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      try {
+        if (currentReader != null && currentReader.nextKeyValue()) {
+          currentPair = nextPair();
+          return true;
+        } else {
+          while (splitsIterator.hasNext()) {
+            // advance the reader and see if it has records
+            InputSplit nextSplit = splitsIterator.next();
+            @SuppressWarnings("unchecked")
+            RecordReader<K, V> reader =
+                (RecordReader<K, V>) format.createRecordReader(nextSplit, attemptContext);
+            if (currentReader != null) {
+              currentReader.close();
+            }
+            currentReader = reader;
+            currentReader.initialize(nextSplit, attemptContext);
+            if (currentReader.nextKeyValue()) {
+              currentPair = nextPair();
+              return true;
+            }
+            currentReader.close();
+            currentReader = null;
+          }
+          // either no next split or all readers were empty
+          currentPair = null;
+          return false;
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IOException(e);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    private 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
+      if (key instanceof Writable) {
+        key = (K) WritableUtils.clone((Writable) key, conf);
+      }
+      if (value instanceof Writable) {
+        value = (V) WritableUtils.clone((Writable) value, conf);
+      }
+      return KV.of(key, value);
+    }
+
+    @Override
+    public KV<K, V> getCurrent() throws NoSuchElementException {
+      if (currentPair == null) {
+        throw new NoSuchElementException();
+      }
+      return currentPair;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (currentReader != null) {
+        currentReader.close();
+        currentReader = null;
+      }
+      currentPair = null;
+    }
+
+    @Override
+    public BoundedSource<KV<K, V>> getCurrentSource() {
+      return source;
+    }
+
+    // BoundedReader
+
+    @Override
+    public Double getFractionConsumed() {
+      if (currentReader == null) {
+        return 0.0;
+      }
+      if (splits.isEmpty()) {
+        return 1.0;
+      }
+      int index = splitsIterator.previousIndex();
+      int numReaders = splits.size();
+      if (index == numReaders) {
+        return 1.0;
+      }
+      double before = 1.0 * index / numReaders;
+      double after = 1.0 * (index + 1) / numReaders;
+      Double fractionOfCurrentReader = getProgress();
+      if (fractionOfCurrentReader == null) {
+        return before;
+      }
+      return before + fractionOfCurrentReader * (after - before);
+    }
+
+    private Double getProgress() {
+      try {
+        return (double) currentReader.getProgress();
+      } catch (IOException | InterruptedException e) {
+        return null;
+      }
+    }
+
+    @Override
+    public BoundedSource<KV<K, V>> splitAtFraction(double fraction) {
+      // Not yet supported. To implement this, the sizes of the splits should be used to
+      // calculate the remaining splits that constitute the given fraction, then a
+      // new source backed by those splits should be returned.
+      return null;
+    }
+  }
+
+  /**
+   * A wrapper to allow Hadoop {@link org.apache.hadoop.mapreduce.InputSplit}s to be
+   * serialized using Java's standard serialization mechanisms. Note that the InputSplit
+   * has to be Writable (which most are).
+   */
+  public static class SerializableSplit implements Externalizable {
+    private static final long serialVersionUID = 0L;
+
+    private InputSplit split;
+
+    public SerializableSplit() {
+    }
+
+    public SerializableSplit(InputSplit split) {
+      Preconditions.checkArgument(split instanceof Writable, "Split is not writable: "
+          + split);
+      this.split = split;
+    }
+
+    public InputSplit getSplit() {
+      return split;
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+      out.writeUTF(split.getClass().getCanonicalName());
+      ((Writable) split).write(out);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+      String className = in.readUTF();
+      try {
+        split = (InputSplit) Class.forName(className).newInstance();
+        ((Writable) split).readFields(in);
+      } catch (InstantiationException | IllegalAccessException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
new file mode 100644
index 0000000..572c24b
--- /dev/null
+++ b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
@@ -0,0 +1,111 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.hadoop;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderException;
+import com.google.cloud.dataflow.sdk.coders.StandardCoder;
+import com.google.cloud.dataflow.sdk.util.CloudObject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.io.Writable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * A {@code WritableCoder} is a {@link com.google.cloud.dataflow.sdk.coders.Coder} for a
+ * Java class that implements {@link org.apache.hadoop.io.Writable}.
+ *
+ * <p> To use, specify the coder type on a PCollection:
+ * <pre>
+ * {@code
+ *   PCollection<MyRecord> records =
+ *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
+ * }
+ * </pre>
+ *
+ * @param <T> the type of elements handled by this coder
+ */
+public class WritableCoder<T extends Writable> extends StandardCoder<T> {
+  private static final long serialVersionUID = 0L;
+
+  /**
+   * Returns a {@code WritableCoder} instance for the provided element class.
+   * @param <T> the element type
+   */
+  public static <T extends Writable> WritableCoder<T> of(Class<T> clazz) {
+    return new WritableCoder<>(clazz);
+  }
+
+  @JsonCreator
+  @SuppressWarnings("unchecked")
+  public static WritableCoder<?> of(@JsonProperty("type") String classType)
+      throws ClassNotFoundException {
+    Class<?> clazz = Class.forName(classType);
+    if (!Writable.class.isAssignableFrom(clazz)) {
+      throw new ClassNotFoundException(
+          "Class " + classType + " does not implement Writable");
+    }
+    return of((Class<? extends Writable>) clazz);
+  }
+
+  private final Class<T> type;
+
+  public WritableCoder(Class<T> type) {
+    this.type = type;
+  }
+
+  @Override
+  public void encode(T value, OutputStream outStream, Context context) throws IOException {
+    value.write(new DataOutputStream(outStream));
+  }
+
+  @Override
+  public T decode(InputStream inStream, Context context) throws IOException {
+    try {
+      T t = type.newInstance();
+      t.readFields(new DataInputStream(inStream));
+      return t;
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new CoderException("unable to deserialize record", e);
+    }
+  }
+
+  @Override
+  public List<Coder<?>> getCoderArguments() {
+    return null;
+  }
+
+  @Override
+  public CloudObject asCloudObject() {
+    CloudObject result = super.asCloudObject();
+    result.put("type", type.getName());
+    return result;
+  }
+
+  @Override
+  public void verifyDeterministic() throws NonDeterministicException {
+    throw new NonDeterministicException(this,
+        "Hadoop Writable may be non-deterministic.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSourceTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSourceTest.java b/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSourceTest.java
deleted file mode 100644
index 4aeb0f8..0000000
--- a/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/HadoopFileSourceTest.java
+++ /dev/null
@@ -1,190 +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 com.google.cloud.dataflow.contrib.hadoop;
-
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-/**
- * Tests for HadoopFileSource.
- */
-public class HadoopFileSourceTest {
-
-  Random random = new Random(0L);
-
-  @Rule
-  public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Test
-  public void testFullyReadSingleFile() throws Exception {
-    PipelineOptions options = PipelineOptionsFactory.create();
-    List<KV<IntWritable, Text>> expectedResults = createRandomRecords(3, 10, 0);
-    File file = createFileWithData("tmp.seq", expectedResults);
-
-    HadoopFileSource<IntWritable, Text> source =
-        HadoopFileSource.from(file.toString(), SequenceFileInputFormat.class,
-            IntWritable.class, Text.class);
-
-    assertEquals(file.length(), source.getEstimatedSizeBytes(null));
-
-    assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray()));
-  }
-
-  @Test
-  public void testFullyReadFilePattern() throws IOException {
-    PipelineOptions options = PipelineOptionsFactory.create();
-    List<KV<IntWritable, Text>> data1 = createRandomRecords(3, 10, 0);
-    File file1 = createFileWithData("file1", data1);
-
-    List<KV<IntWritable, Text>> data2 = createRandomRecords(3, 10, 10);
-    createFileWithData("file2", data2);
-
-    List<KV<IntWritable, Text>> data3 = createRandomRecords(3, 10, 20);
-    createFileWithData("file3", data3);
-
-    List<KV<IntWritable, Text>> data4 = createRandomRecords(3, 10, 30);
-    createFileWithData("otherfile", data4);
-
-    HadoopFileSource<IntWritable, Text> source =
-        HadoopFileSource.from(new File(file1.getParent(), "file*").toString(),
-            SequenceFileInputFormat.class, IntWritable.class, Text.class);
-    List<KV<IntWritable, Text>> expectedResults = new ArrayList<>();
-    expectedResults.addAll(data1);
-    expectedResults.addAll(data2);
-    expectedResults.addAll(data3);
-    assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray()));
-  }
-
-  @Test
-  public void testCloseUnstartedFilePatternReader() throws IOException {
-    PipelineOptions options = PipelineOptionsFactory.create();
-    List<KV<IntWritable, Text>> data1 = createRandomRecords(3, 10, 0);
-    File file1 = createFileWithData("file1", data1);
-
-    List<KV<IntWritable, Text>> data2 = createRandomRecords(3, 10, 10);
-    createFileWithData("file2", data2);
-
-    List<KV<IntWritable, Text>> data3 = createRandomRecords(3, 10, 20);
-    createFileWithData("file3", data3);
-
-    List<KV<IntWritable, Text>> data4 = createRandomRecords(3, 10, 30);
-    createFileWithData("otherfile", data4);
-
-    HadoopFileSource<IntWritable, Text> source =
-        HadoopFileSource.from(new File(file1.getParent(), "file*").toString(),
-            SequenceFileInputFormat.class, IntWritable.class, Text.class);
-    Source.Reader<KV<IntWritable, Text>> reader = source.createReader(options);
-    // Closing an unstarted FilePatternReader should not throw an exception.
-    try {
-      reader.close();
-    } catch (Exception e) {
-      fail("Closing an unstarted FilePatternReader should not throw an exception");
-    }
-  }
-
-  @Test
-  public void testSplits() throws Exception {
-    PipelineOptions options = PipelineOptionsFactory.create();
-
-    List<KV<IntWritable, Text>> expectedResults = createRandomRecords(3, 10000, 0);
-    File file = createFileWithData("tmp.avro", expectedResults);
-
-    HadoopFileSource<IntWritable, Text> source =
-        HadoopFileSource.from(file.toString(), SequenceFileInputFormat.class,
-            IntWritable.class, Text.class);
-
-    // Assert that the source produces the expected records
-    assertEquals(expectedResults, readFromSource(source, options));
-
-    // Split with a small bundle size (has to be at least size of sync interval)
-    List<? extends BoundedSource<KV<IntWritable, Text>>> splits = source
-        .splitIntoBundles(SequenceFile.SYNC_INTERVAL, options);
-    assertTrue(splits.size() > 2);
-    SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options);
-    int nonEmptySplits = 0;
-    for (BoundedSource<KV<IntWritable, Text>> subSource : splits) {
-      if (readFromSource(subSource, options).size() > 0) {
-        nonEmptySplits += 1;
-      }
-    }
-    assertTrue(nonEmptySplits > 2);
-  }
-
-  private File createFileWithData(String filename, List<KV<IntWritable, Text>> records)
-      throws IOException {
-    File tmpFile = tmpFolder.newFile(filename);
-    try (Writer writer = SequenceFile.createWriter(new Configuration(),
-          Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class),
-          Writer.file(new Path(tmpFile.toURI())))) {
-
-      for (KV<IntWritable, Text> record : records) {
-        writer.append(record.getKey(), record.getValue());
-      }
-    }
-    return tmpFile;
-  }
-
-  private List<KV<IntWritable, Text>> createRandomRecords(int dataItemLength,
-      int numItems, int offset) {
-    List<KV<IntWritable, Text>> records = new ArrayList<>();
-    for (int i = 0; i < numItems; i++) {
-      IntWritable key = new IntWritable(i + offset);
-      Text value = new Text(createRandomString(dataItemLength));
-      records.add(KV.of(key, value));
-    }
-    return records;
-  }
-
-  private String createRandomString(int length) {
-    char[] chars = "abcdefghijklmnopqrstuvwxyz".toCharArray();
-    StringBuilder builder = new StringBuilder();
-    for (int i = 0; i < length; i++) {
-      builder.append(chars[random.nextInt(chars.length)]);
-    }
-    return builder.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoderTest.java b/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoderTest.java
deleted file mode 100644
index fac3fe2..0000000
--- a/contrib/hadoop/src/test/java/com/google/cloud/dataflow/contrib/hadoop/WritableCoderTest.java
+++ /dev/null
@@ -1,37 +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 com.google.cloud.dataflow.contrib.hadoop;
-
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-
-import org.apache.hadoop.io.IntWritable;
-import org.junit.Test;
-
-/**
- * Tests for WritableCoder.
- */
-public class WritableCoderTest {
-
-  @Test
-  public void testIntWritableEncoding() throws Exception {
-    IntWritable value = new IntWritable(42);
-    WritableCoder<IntWritable> coder = WritableCoder.of(IntWritable.class);
-
-    CoderProperties.coderDecodeEncodeEqual(coder, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
new file mode 100644
index 0000000..4aeb0f8
--- /dev/null
+++ b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.hadoop;
+
+import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Source;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
+import com.google.cloud.dataflow.sdk.values.KV;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Tests for HadoopFileSource.
+ */
+public class HadoopFileSourceTest {
+
+  Random random = new Random(0L);
+
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testFullyReadSingleFile() throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    List<KV<IntWritable, Text>> expectedResults = createRandomRecords(3, 10, 0);
+    File file = createFileWithData("tmp.seq", expectedResults);
+
+    HadoopFileSource<IntWritable, Text> source =
+        HadoopFileSource.from(file.toString(), SequenceFileInputFormat.class,
+            IntWritable.class, Text.class);
+
+    assertEquals(file.length(), source.getEstimatedSizeBytes(null));
+
+    assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray()));
+  }
+
+  @Test
+  public void testFullyReadFilePattern() throws IOException {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    List<KV<IntWritable, Text>> data1 = createRandomRecords(3, 10, 0);
+    File file1 = createFileWithData("file1", data1);
+
+    List<KV<IntWritable, Text>> data2 = createRandomRecords(3, 10, 10);
+    createFileWithData("file2", data2);
+
+    List<KV<IntWritable, Text>> data3 = createRandomRecords(3, 10, 20);
+    createFileWithData("file3", data3);
+
+    List<KV<IntWritable, Text>> data4 = createRandomRecords(3, 10, 30);
+    createFileWithData("otherfile", data4);
+
+    HadoopFileSource<IntWritable, Text> source =
+        HadoopFileSource.from(new File(file1.getParent(), "file*").toString(),
+            SequenceFileInputFormat.class, IntWritable.class, Text.class);
+    List<KV<IntWritable, Text>> expectedResults = new ArrayList<>();
+    expectedResults.addAll(data1);
+    expectedResults.addAll(data2);
+    expectedResults.addAll(data3);
+    assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray()));
+  }
+
+  @Test
+  public void testCloseUnstartedFilePatternReader() throws IOException {
+    PipelineOptions options = PipelineOptionsFactory.create();
+    List<KV<IntWritable, Text>> data1 = createRandomRecords(3, 10, 0);
+    File file1 = createFileWithData("file1", data1);
+
+    List<KV<IntWritable, Text>> data2 = createRandomRecords(3, 10, 10);
+    createFileWithData("file2", data2);
+
+    List<KV<IntWritable, Text>> data3 = createRandomRecords(3, 10, 20);
+    createFileWithData("file3", data3);
+
+    List<KV<IntWritable, Text>> data4 = createRandomRecords(3, 10, 30);
+    createFileWithData("otherfile", data4);
+
+    HadoopFileSource<IntWritable, Text> source =
+        HadoopFileSource.from(new File(file1.getParent(), "file*").toString(),
+            SequenceFileInputFormat.class, IntWritable.class, Text.class);
+    Source.Reader<KV<IntWritable, Text>> reader = source.createReader(options);
+    // Closing an unstarted FilePatternReader should not throw an exception.
+    try {
+      reader.close();
+    } catch (Exception e) {
+      fail("Closing an unstarted FilePatternReader should not throw an exception");
+    }
+  }
+
+  @Test
+  public void testSplits() throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.create();
+
+    List<KV<IntWritable, Text>> expectedResults = createRandomRecords(3, 10000, 0);
+    File file = createFileWithData("tmp.avro", expectedResults);
+
+    HadoopFileSource<IntWritable, Text> source =
+        HadoopFileSource.from(file.toString(), SequenceFileInputFormat.class,
+            IntWritable.class, Text.class);
+
+    // Assert that the source produces the expected records
+    assertEquals(expectedResults, readFromSource(source, options));
+
+    // Split with a small bundle size (has to be at least size of sync interval)
+    List<? extends BoundedSource<KV<IntWritable, Text>>> splits = source
+        .splitIntoBundles(SequenceFile.SYNC_INTERVAL, options);
+    assertTrue(splits.size() > 2);
+    SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options);
+    int nonEmptySplits = 0;
+    for (BoundedSource<KV<IntWritable, Text>> subSource : splits) {
+      if (readFromSource(subSource, options).size() > 0) {
+        nonEmptySplits += 1;
+      }
+    }
+    assertTrue(nonEmptySplits > 2);
+  }
+
+  private File createFileWithData(String filename, List<KV<IntWritable, Text>> records)
+      throws IOException {
+    File tmpFile = tmpFolder.newFile(filename);
+    try (Writer writer = SequenceFile.createWriter(new Configuration(),
+          Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class),
+          Writer.file(new Path(tmpFile.toURI())))) {
+
+      for (KV<IntWritable, Text> record : records) {
+        writer.append(record.getKey(), record.getValue());
+      }
+    }
+    return tmpFile;
+  }
+
+  private List<KV<IntWritable, Text>> createRandomRecords(int dataItemLength,
+      int numItems, int offset) {
+    List<KV<IntWritable, Text>> records = new ArrayList<>();
+    for (int i = 0; i < numItems; i++) {
+      IntWritable key = new IntWritable(i + offset);
+      Text value = new Text(createRandomString(dataItemLength));
+      records.add(KV.of(key, value));
+    }
+    return records;
+  }
+
+  private String createRandomString(int length) {
+    char[] chars = "abcdefghijklmnopqrstuvwxyz".toCharArray();
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < length; i++) {
+      builder.append(chars[random.nextInt(chars.length)]);
+    }
+    return builder.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
new file mode 100644
index 0000000..fac3fe2
--- /dev/null
+++ b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.hadoop;
+
+import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+
+import org.apache.hadoop.io.IntWritable;
+import org.junit.Test;
+
+/**
+ * Tests for WritableCoder.
+ */
+public class WritableCoderTest {
+
+  @Test
+  public void testIntWritableEncoding() throws Exception {
+    IntWritable value = new IntWritable(42);
+    WritableCoder<IntWritable> coder = WritableCoder.of(IntWritable.class);
+
+    CoderProperties.coderDecodeEncodeEqual(coder, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java b/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java
deleted file mode 100644
index b7f0df7..0000000
--- a/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java
+++ /dev/null
@@ -1,186 +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 com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Preconditions;
-
-/**
- * Utility class with different versions of joins. All methods join two collections of
- * key/value pairs (KV).
- */
-public class Join {
-
-  /**
-   * Inner join of two collections of KV elements.
-   * @param leftCollection Left side collection to join.
-   * @param rightCollection Right side collection to join.
-   * @param <K> Type of the key for both collections
-   * @param <V1> Type of the values for the left collection.
-   * @param <V2> Type of the values for the right collection.
-   * @return A joined collection of KV where Key is the key and value is a
-   *         KV where Key is of type V1 and Value is type V2.
-   */
-  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);
-
-    final TupleTag<V1> v1Tuple = new TupleTag<>();
-    final TupleTag<V2> v2Tuple = new TupleTag<>();
-
-    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
-      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
-        .and(v2Tuple, rightCollection)
-        .apply(CoGroupByKey.<K>create());
-
-    return coGbkResultCollection.apply(ParDo.of(
-      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
-        @Override
-        public void processElement(ProcessContext c) {
-          KV<K, CoGbkResult> e = c.element();
-
-          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
-          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
-
-          for (V1 leftValue : leftValuesIterable) {
-            for (V2 rightValue : rightValuesIterable) {
-              c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
-            }
-          }
-        }
-      }))
-      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
-                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
-                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
-  }
-
-  /**
-   * Left Outer Join of two collections of KV elements.
-   * @param leftCollection Left side collection to join.
-   * @param rightCollection Right side collection to join.
-   * @param nullValue Value to use as null value when right side do not match left side.
-   * @param <K> Type of the key for both collections
-   * @param <V1> Type of the values for the left collection.
-   * @param <V2> Type of the values for the right collection.
-   * @return A joined collection of KV where Key is the key and value is a
-   *         KV where Key is of type V1 and Value is type V2. Values that
-   *         should be null or empty is replaced with nullValue.
-   */
-  public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> leftOuterJoin(
-    final PCollection<KV<K, V1>> leftCollection,
-    final PCollection<KV<K, V2>> rightCollection,
-    final V2 nullValue) {
-    Preconditions.checkNotNull(leftCollection);
-    Preconditions.checkNotNull(rightCollection);
-    Preconditions.checkNotNull(nullValue);
-
-    final TupleTag<V1> v1Tuple = new TupleTag<>();
-    final TupleTag<V2> v2Tuple = new TupleTag<>();
-
-    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
-      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
-        .and(v2Tuple, rightCollection)
-        .apply(CoGroupByKey.<K>create());
-
-    return coGbkResultCollection.apply(ParDo.of(
-      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
-        @Override
-        public void processElement(ProcessContext c) {
-          KV<K, CoGbkResult> e = c.element();
-
-          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
-          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
-
-          for (V1 leftValue : leftValuesIterable) {
-            if (rightValuesIterable.iterator().hasNext()) {
-              for (V2 rightValue : rightValuesIterable) {
-                c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
-              }
-            } else {
-              c.output(KV.of(e.getKey(), KV.of(leftValue, nullValue)));
-            }
-          }
-        }
-      }))
-      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
-                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
-                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
-  }
-
-  /**
-   * Right Outer Join of two collections of KV elements.
-   * @param leftCollection Left side collection to join.
-   * @param rightCollection Right side collection to join.
-   * @param nullValue Value to use as null value when left side do not match right side.
-   * @param <K> Type of the key for both collections
-   * @param <V1> Type of the values for the left collection.
-   * @param <V2> Type of the values for the right collection.
-   * @return A joined collection of KV where Key is the key and value is a
-   *         KV where Key is of type V1 and Value is type V2. Keys that
-   *         should be null or empty is replaced with nullValue.
-   */
-  public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> rightOuterJoin(
-    final PCollection<KV<K, V1>> leftCollection,
-    final PCollection<KV<K, V2>> rightCollection,
-    final V1 nullValue) {
-    Preconditions.checkNotNull(leftCollection);
-    Preconditions.checkNotNull(rightCollection);
-    Preconditions.checkNotNull(nullValue);
-
-    final TupleTag<V1> v1Tuple = new TupleTag<>();
-    final TupleTag<V2> v2Tuple = new TupleTag<>();
-
-    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
-      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
-        .and(v2Tuple, rightCollection)
-        .apply(CoGroupByKey.<K>create());
-
-    return coGbkResultCollection.apply(ParDo.of(
-      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
-        @Override
-        public void processElement(ProcessContext c) {
-          KV<K, CoGbkResult> e = c.element();
-
-          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
-          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
-
-          for (V2 rightValue : rightValuesIterable) {
-            if (leftValuesIterable.iterator().hasNext()) {
-              for (V1 leftValue : leftValuesIterable) {
-                c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
-              }
-            } else {
-              c.output(KV.of(e.getKey(), KV.of(nullValue, rightValue)));
-            }
-          }
-        }
-      }))
-      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
-                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
-                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java b/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
new file mode 100644
index 0000000..b7f0df7
--- /dev/null
+++ b/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
@@ -0,0 +1,186 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.joinlibrary;
+
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility class with different versions of joins. All methods join two collections of
+ * key/value pairs (KV).
+ */
+public class Join {
+
+  /**
+   * Inner join of two collections of KV elements.
+   * @param leftCollection Left side collection to join.
+   * @param rightCollection Right side collection to join.
+   * @param <K> Type of the key for both collections
+   * @param <V1> Type of the values for the left collection.
+   * @param <V2> Type of the values for the right collection.
+   * @return A joined collection of KV where Key is the key and value is a
+   *         KV where Key is of type V1 and Value is type V2.
+   */
+  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);
+
+    final TupleTag<V1> v1Tuple = new TupleTag<>();
+    final TupleTag<V2> v2Tuple = new TupleTag<>();
+
+    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
+      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
+        .and(v2Tuple, rightCollection)
+        .apply(CoGroupByKey.<K>create());
+
+    return coGbkResultCollection.apply(ParDo.of(
+      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
+        @Override
+        public void processElement(ProcessContext c) {
+          KV<K, CoGbkResult> e = c.element();
+
+          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
+          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
+
+          for (V1 leftValue : leftValuesIterable) {
+            for (V2 rightValue : rightValuesIterable) {
+              c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
+            }
+          }
+        }
+      }))
+      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
+                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
+                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
+  }
+
+  /**
+   * Left Outer Join of two collections of KV elements.
+   * @param leftCollection Left side collection to join.
+   * @param rightCollection Right side collection to join.
+   * @param nullValue Value to use as null value when right side do not match left side.
+   * @param <K> Type of the key for both collections
+   * @param <V1> Type of the values for the left collection.
+   * @param <V2> Type of the values for the right collection.
+   * @return A joined collection of KV where Key is the key and value is a
+   *         KV where Key is of type V1 and Value is type V2. Values that
+   *         should be null or empty is replaced with nullValue.
+   */
+  public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> leftOuterJoin(
+    final PCollection<KV<K, V1>> leftCollection,
+    final PCollection<KV<K, V2>> rightCollection,
+    final V2 nullValue) {
+    Preconditions.checkNotNull(leftCollection);
+    Preconditions.checkNotNull(rightCollection);
+    Preconditions.checkNotNull(nullValue);
+
+    final TupleTag<V1> v1Tuple = new TupleTag<>();
+    final TupleTag<V2> v2Tuple = new TupleTag<>();
+
+    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
+      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
+        .and(v2Tuple, rightCollection)
+        .apply(CoGroupByKey.<K>create());
+
+    return coGbkResultCollection.apply(ParDo.of(
+      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
+        @Override
+        public void processElement(ProcessContext c) {
+          KV<K, CoGbkResult> e = c.element();
+
+          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
+          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
+
+          for (V1 leftValue : leftValuesIterable) {
+            if (rightValuesIterable.iterator().hasNext()) {
+              for (V2 rightValue : rightValuesIterable) {
+                c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
+              }
+            } else {
+              c.output(KV.of(e.getKey(), KV.of(leftValue, nullValue)));
+            }
+          }
+        }
+      }))
+      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
+                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
+                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
+  }
+
+  /**
+   * Right Outer Join of two collections of KV elements.
+   * @param leftCollection Left side collection to join.
+   * @param rightCollection Right side collection to join.
+   * @param nullValue Value to use as null value when left side do not match right side.
+   * @param <K> Type of the key for both collections
+   * @param <V1> Type of the values for the left collection.
+   * @param <V2> Type of the values for the right collection.
+   * @return A joined collection of KV where Key is the key and value is a
+   *         KV where Key is of type V1 and Value is type V2. Keys that
+   *         should be null or empty is replaced with nullValue.
+   */
+  public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> rightOuterJoin(
+    final PCollection<KV<K, V1>> leftCollection,
+    final PCollection<KV<K, V2>> rightCollection,
+    final V1 nullValue) {
+    Preconditions.checkNotNull(leftCollection);
+    Preconditions.checkNotNull(rightCollection);
+    Preconditions.checkNotNull(nullValue);
+
+    final TupleTag<V1> v1Tuple = new TupleTag<>();
+    final TupleTag<V2> v2Tuple = new TupleTag<>();
+
+    PCollection<KV<K, CoGbkResult>> coGbkResultCollection =
+      KeyedPCollectionTuple.of(v1Tuple, leftCollection)
+        .and(v2Tuple, rightCollection)
+        .apply(CoGroupByKey.<K>create());
+
+    return coGbkResultCollection.apply(ParDo.of(
+      new DoFn<KV<K, CoGbkResult>, KV<K, KV<V1, V2>>>() {
+        @Override
+        public void processElement(ProcessContext c) {
+          KV<K, CoGbkResult> e = c.element();
+
+          Iterable<V1> leftValuesIterable = e.getValue().getAll(v1Tuple);
+          Iterable<V2> rightValuesIterable = e.getValue().getAll(v2Tuple);
+
+          for (V2 rightValue : rightValuesIterable) {
+            if (leftValuesIterable.iterator().hasNext()) {
+              for (V1 leftValue : leftValuesIterable) {
+                c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue)));
+              }
+            } else {
+              c.output(KV.of(e.getKey(), KV.of(nullValue, rightValue)));
+            }
+          }
+        }
+      }))
+      .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(),
+                           KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(),
+                                      ((KvCoder) rightCollection.getCoder()).getValueCoder())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java
deleted file mode 100644
index c479a65..0000000
--- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java
+++ /dev/null
@@ -1,143 +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 com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * This test Inner Join functionality.
- */
-public class InnerJoinTest {
-
-  Pipeline p;
-  List<KV<String, Long>> leftListOfKv;
-  List<KV<String, String>> listRightOfKv;
-  List<KV<String, KV<Long, String>>> expectedResult;
-
-  @Before
-  public void setup() {
-
-    p = TestPipeline.create();
-    leftListOfKv = new ArrayList<>();
-    listRightOfKv = new ArrayList<>();
-
-    expectedResult = new ArrayList<>();
-  }
-
-  @Test
-  public void testJoinOneToOneMapping() {
-    leftListOfKv.add(KV.of("Key1", 5L));
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection =
-        p.apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key1", "foo"));
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection =
-        p.apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
-      leftCollection, rightCollection);
-
-    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinOneToManyMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    listRightOfKv.add(KV.of("Key2", "gazonk"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
-      leftCollection, rightCollection);
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinManyToOneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    leftListOfKv.add(KV.of("Key2", 6L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
-      leftCollection, rightCollection);
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinNoneToNoneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key3", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
-      leftCollection, rightCollection);
-
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-    p.run();
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinLeftCollectionNull() {
-    Join.innerJoin(null, p.apply(Create.of(listRightOfKv)));
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinRightCollectionNull() {
-    Join.innerJoin(p.apply(Create.of(leftListOfKv)), null);
-  }
-}



[13/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java
deleted file mode 100644
index ed25926..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java
+++ /dev/null
@@ -1,991 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.nio.charset.StandardCharsets;
-import java.util.NoSuchElementException;
-import java.util.regex.Pattern;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link PTransform}s for reading and writing text files.
- *
- * <p>To read a {@link PCollection} from one or more text files, use {@link TextIO.Read}.
- * You can instantiate a transform using {@link TextIO.Read#from(String)} 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>"}). You may optionally call
- * {@link TextIO.Read#named(String)} to specify the name of the pipeline step.
- *
- * <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
- * bytes (split into lines delimited by '\n', '\r', or '\r\n') to another object of type {@code T},
- * supply a {@code Coder<T>} using {@link TextIO.Read#withCoder(Coder)}.
- *
- * <p>See the following examples:
- *
- * <pre>{@code
- * Pipeline p = ...;
- *
- * // A simple Read of a local file (only runs locally):
- * PCollection<String> lines =
- *     p.apply(TextIO.Read.from("/local/path/to/file.txt"));
- *
- * // 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()));
- * }</pre>
- *
- * <p>To write a {@link PCollection} to one or more text files, use
- * {@link TextIO.Write}, specifying {@link TextIO.Write#to(String)} to specify
- * 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.
- *
- * <p>Any existing files with the same names as generated output files
- * will be overwritten.
- *
- * <p>For example:
- * <pre>{@code
- * // A simple Write to a local file (only runs locally):
- * PCollection<String> lines = ...;
- * lines.apply(TextIO.Write.to("/path/to/file.txt"));
- *
- * // 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()));
- * }</pre>
- *
- * <h3>Permissions</h3>
- * <p>When run using the {@link DirectPipelineRunner}, 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
- * documentation on <a href="https://cloud.google.com/dataflow/security-and-permissions">Security
- * and Permissions</a>.
- */
-public class TextIO {
-  /** The default coder, which returns each line of the input file as a string. */
-  public static final Coder<String> DEFAULT_TEXT_CODER = StringUtf8Coder.of();
-
-  /**
-   * A {@link PTransform} that reads from a text file (or multiple text
-   * files matching a pattern) and returns a {@link PCollection} containing
-   * the decoding of each of the lines of the text file(s). The
-   * default decoding just returns each line as a {@link String}, but you may call
-   * {@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)
-     * with the given filename or filename pattern. This can be a local path (if running locally),
-     * or a Google Cloud Storage filename or filename pattern of the form
-     * {@code "gs://<bucket>/<filepath>"} (if running locally or via the Google Cloud Dataflow
-     * service). Standard <a href="http://docs.oracle.com/javase/tutorial/essential/io/find.html"
-     * >Java Filesystem glob patterns</a> ("*", "?", "[..]") are supported.
-     */
-    public static Bound<String> from(String filepattern) {
-      return new Bound<>(DEFAULT_TEXT_CODER).from(filepattern);
-    }
-
-    /**
-     * Returns a transform for reading text files that uses the given
-     * {@code Coder<T>} to decode each of the lines of the file into a
-     * value of type {@code T}.
-     *
-     * <p>By default, uses {@link StringUtf8Coder}, which just
-     * returns the text lines as Java strings.
-     *
-     * @param <T> the type of the decoded elements, and the elements
-     * of the resulting PCollection
-     */
-    public static <T> Bound<T> withCoder(Coder<T> coder) {
-      return new Bound<>(coder);
-    }
-
-    /**
-     * Returns a transform for reading text files that has GCS path validation on
-     * pipeline creation disabled.
-     *
-     * <p>This can be useful in the case where the GCS input does not
-     * exist at the pipeline creation time, but is expected to be
-     * available at execution time.
-     */
-    public static Bound<String> withoutValidation() {
-      return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation();
-    }
-
-    /**
-     * Returns a transform for reading text files that decompresses all input files
-     * using the specified compression type.
-     *
-     * <p>If no compression type is specified, the default is {@link TextIO.CompressionType#AUTO}.
-     * In this mode, the compression type of the file is determined by its extension
-     * (e.g., {@code *.gz} is gzipped, {@code *.bz2} is bzipped, and all other extensions are
-     * uncompressed).
-     */
-    public static Bound<String> withCompressionType(TextIO.CompressionType compressionType) {
-      return new Bound<>(DEFAULT_TEXT_CODER).withCompressionType(compressionType);
-    }
-
-    // TODO: strippingNewlines, etc.
-
-    /**
-     * A {@link PTransform} that reads from one or more text files and returns a bounded
-     * {@link PCollection} containing one element for each line of the input files.
-     *
-     * @param <T> the type of each of the elements of the resulting
-     * {@link PCollection}. By default, each line is returned as a {@link String}, however you
-     * may use {@link #withCoder(Coder)} to supply a {@code Coder<T>} to produce a
-     * {@code PCollection<T>} instead.
-     */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
-      /** The filepattern to read from. */
-      @Nullable private final String filepattern;
-
-      /** The Coder to use to decode each line. */
-      private final Coder<T> coder;
-
-      /** An option to indicate if input validation is desired. Default is true. */
-      private final boolean validate;
-
-      /** Option to indicate the input source's compression type. Default is AUTO. */
-      private final TextIO.CompressionType compressionType;
-
-      Bound(Coder<T> coder) {
-        this(null, null, coder, true, TextIO.CompressionType.AUTO);
-      }
-
-      private Bound(String name, String filepattern, Coder<T> coder, boolean validate,
-          TextIO.CompressionType compressionType) {
-        super(name);
-        this.coder = coder;
-        this.filepattern = filepattern;
-        this.validate = validate;
-        this.compressionType = compressionType;
-      }
-
-      /**
-       * 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.
-       *
-       * <p>Does not modify this object.
-
-       */
-      public Bound<T> from(String filepattern) {
-        return new Bound<>(name, filepattern, coder, validate, compressionType);
-      }
-
-      /**
-       * Returns a new transform for reading from text files that's like this one but
-       * that uses the given {@link Coder Coder<X>} to decode each of the
-       * lines of the file into a value of type {@code X}.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param <X> the type of the decoded elements, and the
-       * elements of the resulting PCollection
-       */
-      public <X> Bound<X> withCoder(Coder<X> coder) {
-        return new Bound<>(name, filepattern, coder, validate, compressionType);
-      }
-
-      /**
-       * Returns a new transform for reading from text files that's like this one but
-       * that has GCS path validation on pipeline creation disabled.
-       *
-       * <p>This can be useful in the case where the GCS input does not
-       * exist at the pipeline creation time, but is expected to be
-       * available at execution time.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withoutValidation() {
-        return new Bound<>(name, filepattern, coder, false, compressionType);
-      }
-
-      /**
-       * Returns a new transform for reading from text files that's like this one but
-       * reads from input sources using the specified compression type.
-       *
-       * <p>If no compression type is specified, the default is {@link TextIO.CompressionType#AUTO}.
-       * See {@link TextIO.Read#withCompressionType} for more details.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withCompressionType(TextIO.CompressionType compressionType) {
-        return new Bound<>(name, filepattern, coder, validate, compressionType);
-      }
-
-      @Override
-      public PCollection<T> apply(PInput input) {
-        if (filepattern == null) {
-          throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform");
-        }
-
-        if (validate) {
-          try {
-            checkState(
-                !IOChannelUtils.getFactory(filepattern).match(filepattern).isEmpty(),
-                "Unable to find any files matching %s",
-                filepattern);
-          } catch (IOException e) {
-            throw new IllegalStateException(
-                String.format("Failed to validate %s", filepattern), e);
-          }
-        }
-
-        // Create a source specific to the requested compression type.
-        final Bounded<T> read;
-        switch(compressionType) {
-          case UNCOMPRESSED:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
-                new TextSource<T>(filepattern, coder));
-            break;
-          case AUTO:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
-                CompressedSource.from(new TextSource<T>(filepattern, coder)));
-            break;
-          case BZIP2:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
-                CompressedSource.from(new TextSource<T>(filepattern, coder))
-                                .withDecompression(CompressedSource.CompressionMode.BZIP2));
-            break;
-          case GZIP:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
-                CompressedSource.from(new TextSource<T>(filepattern, coder))
-                                .withDecompression(CompressedSource.CompressionMode.GZIP));
-            break;
-          default:
-            throw new IllegalArgumentException("Unknown compression mode: " + compressionType);
-        }
-
-        PCollection<T> pcol = input.getPipeline().apply("Read", read);
-        // Honor the default output coder that would have been used by this PTransform.
-        pcol.setCoder(getDefaultOutputCoder());
-        return pcol;
-      }
-
-      @Override
-      protected Coder<T> getDefaultOutputCoder() {
-        return coder;
-      }
-
-      public String getFilepattern() {
-        return filepattern;
-      }
-
-      public boolean needsValidation() {
-        return validate;
-      }
-
-      public TextIO.CompressionType getCompressionType() {
-        return compressionType;
-      }
-    }
-
-    /** Disallow construction of utility classes. */
-    private Read() {}
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * A {@link PTransform} that writes a {@link PCollection} to text file (or
-   * multiple text files matching a sharding pattern), with each
-   * 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)
-     * with the given prefix. This can be a local filename
-     * (if running locally), or a Google Cloud Storage filename of
-     * the form {@code "gs://<bucket>/<filepath>"}
-     * (if running locally or via the Google Cloud Dataflow service).
-     *
-     * <p>The files written will begin with this prefix, followed by
-     * a shard identifier (see {@link Bound#withNumShards(int)}, and end
-     * in a common extension, if given by {@link Bound#withSuffix(String)}.
-     */
-    public static Bound<String> to(String prefix) {
-      return new Bound<>(DEFAULT_TEXT_CODER).to(prefix);
-    }
-
-    /**
-     * Returns a transform for writing to text files that appends the specified suffix
-     * to the created files.
-     */
-    public static Bound<String> withSuffix(String nameExtension) {
-      return new Bound<>(DEFAULT_TEXT_CODER).withSuffix(nameExtension);
-    }
-
-    /**
-     * Returns a transform for writing to text files that uses the provided shard count.
-     *
-     * <p>Constraining the number of shards is likely to reduce
-     * the performance of a pipeline. Setting this value is not recommended
-     * unless you require a specific number of output files.
-     *
-     * @param numShards the number of shards to use, or 0 to let the system
-     *                  decide.
-     */
-    public static Bound<String> withNumShards(int numShards) {
-      return new Bound<>(DEFAULT_TEXT_CODER).withNumShards(numShards);
-    }
-
-    /**
-     * Returns a transform for writing to text files that uses the given shard name
-     * template.
-     *
-     * <p>See {@link ShardNameTemplate} for a description of shard templates.
-     */
-    public static Bound<String> withShardNameTemplate(String shardTemplate) {
-      return new Bound<>(DEFAULT_TEXT_CODER).withShardNameTemplate(shardTemplate);
-    }
-
-    /**
-     * Returns a transform for writing to text files that forces a single file as
-     * output.
-     */
-    public static Bound<String> withoutSharding() {
-      return new Bound<>(DEFAULT_TEXT_CODER).withoutSharding();
-    }
-
-    /**
-     * Returns a transform for writing to text files that uses the given
-     * {@link Coder} to encode each of the elements of the input
-     * {@link PCollection} into an output text line.
-     *
-     * <p>By default, uses {@link StringUtf8Coder}, which writes input
-     * Java strings directly as output lines.
-     *
-     * @param <T> the type of the elements of the input {@link PCollection}
-     */
-    public static <T> Bound<T> withCoder(Coder<T> coder) {
-      return new Bound<>(coder);
-    }
-
-    /**
-     * Returns a transform for writing to text files that has GCS path validation on
-     * pipeline creation disabled.
-     *
-     * <p>This can be useful in the case where the GCS output location does
-     * not exist at the pipeline creation time, but is expected to be available
-     * at execution time.
-     */
-    public static Bound<String> withoutValidation() {
-      return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation();
-    }
-
-    // TODO: appendingNewlines, header, footer, etc.
-
-    /**
-     * A PTransform that writes a bounded PCollection to a text file (or
-     * multiple text files matching a sharding pattern), with each
-     * PCollection element being encoded into its own line.
-     *
-     * @param <T> the type of the elements of the input PCollection
-     */
-    public static class Bound<T> extends PTransform<PCollection<T>, PDone> {
-      /** The prefix of each file written, combined with suffix and shardTemplate. */
-      @Nullable private final String filenamePrefix;
-      /** The suffix of each file written, combined with prefix and shardTemplate. */
-      private final String filenameSuffix;
-
-      /** The Coder to use to decode each line. */
-      private final Coder<T> coder;
-
-      /** Requested number of shards. 0 for automatic. */
-      private final int numShards;
-
-      /** The shard template of each file written, combined with prefix and suffix. */
-      private final String shardTemplate;
-
-      /** An option to indicate if output validation is desired. Default is true. */
-      private final boolean validate;
-
-      Bound(Coder<T> coder) {
-        this(null, null, "", coder, 0, ShardNameTemplate.INDEX_OF_MAX, true);
-      }
-
-      private Bound(String name, String filenamePrefix, String filenameSuffix, Coder<T> coder,
-          int numShards, String shardTemplate, boolean validate) {
-        super(name);
-        this.coder = coder;
-        this.filenamePrefix = filenamePrefix;
-        this.filenameSuffix = filenameSuffix;
-        this.numShards = numShards;
-        this.shardTemplate = shardTemplate;
-        this.validate = validate;
-      }
-
-      /**
-       * 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.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> to(String filenamePrefix) {
-        validateOutputComponent(filenamePrefix);
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
-            shardTemplate, validate);
-      }
-
-      /**
-       * Returns a transform for writing to text files that that's like this one but
-       * that writes to the file(s) with the given filename suffix.
-       *
-       * <p>Does not modify this object.
-       *
-       * @see ShardNameTemplate
-       */
-      public Bound<T> withSuffix(String nameExtension) {
-        validateOutputComponent(nameExtension);
-        return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards,
-            shardTemplate, validate);
-      }
-
-      /**
-       * Returns a transform for writing to text files that's like this one but
-       * that uses the provided shard count.
-       *
-       * <p>Constraining the number of shards is likely to reduce
-       * the performance of a pipeline. Setting this value is not recommended
-       * unless you require a specific number of output files.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param numShards the number of shards to use, or 0 to let the system
-       *                  decide.
-       * @see ShardNameTemplate
-       */
-      public Bound<T> withNumShards(int numShards) {
-        Preconditions.checkArgument(numShards >= 0);
-        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 uses the given shard name template.
-       *
-       * <p>Does not modify this object.
-       *
-       * @see ShardNameTemplate
-       */
-      public Bound<T> withShardNameTemplate(String shardTemplate) {
-        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 forces a single file as output.
-       *
-       * <p>Constraining the number of shards is likely to reduce
-       * the performance of a pipeline. Using this setting is not recommended
-       * unless you truly require a single output file.
-       *
-       * <p>This is a shortcut for
-       * {@code .withNumShards(1).withShardNameTemplate("")}
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withoutSharding() {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, "", validate);
-      }
-
-      /**
-       * Returns a transform for writing to text files that's like this one
-       * but that uses the given {@link Coder Coder<X>} to encode each of
-       * the elements of the input {@link PCollection PCollection<X>} into an
-       * output text line. Does not modify this object.
-       *
-       * @param <X> the type of the elements of the input {@link PCollection}
-       */
-      public <X> Bound<X> withCoder(Coder<X> coder) {
-        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 has GCS output path validation on pipeline creation disabled.
-       *
-       * <p>This can be useful in the case where the GCS output location does
-       * not exist at the pipeline creation time, but is expected to be
-       * available at execution time.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> withoutValidation() {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
-            shardTemplate, false);
-      }
-
-      @Override
-      public PDone apply(PCollection<T> input) {
-        if (filenamePrefix == null) {
-          throw new IllegalStateException(
-              "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", com.google.cloud.dataflow.sdk.io.Write.to(
-            new TextSink<>(
-                filenamePrefix, filenameSuffix, shardTemplate, coder)));
-      }
-
-      /**
-       * Returns the current shard name template string.
-       */
-      public String getShardNameTemplate() {
-        return shardTemplate;
-      }
-
-      @Override
-      protected Coder<Void> getDefaultOutputCoder() {
-        return VoidCoder.of();
-      }
-
-      public String getFilenamePrefix() {
-        return filenamePrefix;
-      }
-
-      public String getShardTemplate() {
-        return shardTemplate;
-      }
-
-      public int getNumShards() {
-        return numShards;
-      }
-
-      public String getFilenameSuffix() {
-        return filenameSuffix;
-      }
-
-      public Coder<T> getCoder() {
-        return coder;
-      }
-
-      public boolean needsValidation() {
-        return validate;
-      }
-    }
-  }
-
-  /**
-   * Possible text file compression types.
-   */
-  public static enum CompressionType {
-    /**
-     * Automatically determine the compression type based on filename extension.
-     */
-    AUTO(""),
-    /**
-     * Uncompressed (i.e., may be split).
-     */
-    UNCOMPRESSED(""),
-    /**
-     * GZipped.
-     */
-    GZIP(".gz"),
-    /**
-     * BZipped.
-     */
-    BZIP2(".bz2");
-
-    private String filenameSuffix;
-
-    private CompressionType(String suffix) {
-      this.filenameSuffix = suffix;
-    }
-
-    /**
-     * Determine if a given filename matches a compression type based on its extension.
-     * @param filename the filename to match
-     * @return true iff the filename ends with the compression type's known extension.
-     */
-    public boolean matches(String filename) {
-      return filename.toLowerCase().endsWith(filenameSuffix.toLowerCase());
-    }
-  }
-
-  // Pattern which matches old-style shard output patterns, which are now
-  // disallowed.
-  private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)");
-
-  private static void validateOutputComponent(String partialFilePattern) {
-    Preconditions.checkArgument(
-        !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(),
-        "Output name components are not allowed to contain @* or @N patterns: "
-        + partialFilePattern);
-  }
-
-  //////////////////////////////////////////////////////////////////////////////
-
-  /** Disable construction of utility class. */
-  private TextIO() {}
-
-  /**
-   * A {@link FileBasedSource} which can decode records delimited by new line characters.
-   *
-   * <p>This source splits the data into records using {@code UTF-8} {@code \n}, {@code \r}, or
-   * {@code \r\n} as the delimiter. This source is not strict and supports decoding the last record
-   * even if it is not delimited. Finally, no records are decoded if the stream is empty.
-   *
-   * <p>This source supports reading from any arbitrary byte position within the stream. If the
-   * starting position is not {@code 0}, then bytes are skipped until the first delimiter is found
-   * representing the beginning of the first record to be decoded.
-   */
-  @VisibleForTesting
-  static class TextSource<T> extends FileBasedSource<T> {
-    /** The Coder to use to decode each line. */
-    private final Coder<T> coder;
-
-    @VisibleForTesting
-    TextSource(String fileSpec, Coder<T> coder) {
-      super(fileSpec, 1L);
-      this.coder = coder;
-    }
-
-    private TextSource(String fileName, long start, long end, Coder<T> coder) {
-      super(fileName, 1L, start, end);
-      this.coder = coder;
-    }
-
-    @Override
-    protected FileBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) {
-      return new TextSource<>(fileName, start, end, coder);
-    }
-
-    @Override
-    protected FileBasedReader<T> createSingleFileReader(PipelineOptions options) {
-      return new TextBasedReader<>(this);
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-      return false;
-    }
-
-    @Override
-    public Coder<T> getDefaultOutputCoder() {
-      return coder;
-    }
-
-    /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSource.FileBasedReader FileBasedReader}
-     * which can decode records delimited by new line characters.
-     *
-     * See {@link TextSource} for further details.
-     */
-    @VisibleForTesting
-    static class TextBasedReader<T> extends FileBasedReader<T> {
-      private static final int READ_BUFFER_SIZE = 8192;
-      private final Coder<T> coder;
-      private final ByteBuffer readBuffer = ByteBuffer.allocate(READ_BUFFER_SIZE);
-      private ByteString buffer;
-      private int startOfSeparatorInBuffer;
-      private int endOfSeparatorInBuffer;
-      private long startOfNextRecord;
-      private boolean eof;
-      private boolean elementIsPresent;
-      private T currentValue;
-      private ReadableByteChannel inChannel;
-
-      private TextBasedReader(TextSource<T> source) {
-        super(source);
-        coder = source.coder;
-        buffer = ByteString.EMPTY;
-      }
-
-      @Override
-      protected long getCurrentOffset() throws NoSuchElementException {
-        if (!elementIsPresent) {
-          throw new NoSuchElementException();
-        }
-        return startOfNextRecord;
-      }
-
-      @Override
-      public T getCurrent() throws NoSuchElementException {
-        if (!elementIsPresent) {
-          throw new NoSuchElementException();
-        }
-        return currentValue;
-      }
-
-      @Override
-      protected void startReading(ReadableByteChannel channel) throws IOException {
-        this.inChannel = channel;
-        // If the first offset is greater than zero, we need to skip bytes until we see our
-        // first separator.
-        if (getCurrentSource().getStartOffset() > 0) {
-          checkState(channel instanceof SeekableByteChannel,
-              "%s only supports reading from a SeekableByteChannel when given a start offset"
-              + " greater than 0.", TextSource.class.getSimpleName());
-          long requiredPosition = getCurrentSource().getStartOffset() - 1;
-          ((SeekableByteChannel) channel).position(requiredPosition);
-          findSeparatorBounds();
-          buffer = buffer.substring(endOfSeparatorInBuffer);
-          startOfNextRecord = requiredPosition + endOfSeparatorInBuffer;
-          endOfSeparatorInBuffer = 0;
-          startOfSeparatorInBuffer = 0;
-        }
-      }
-
-      /**
-       * Locates the start position and end position of the next delimiter. Will
-       * consume the channel till either EOF or the delimiter bounds are found.
-       *
-       * <p>This fills the buffer and updates the positions as follows:
-       * <pre>{@code
-       * ------------------------------------------------------
-       * | element bytes | delimiter bytes | unconsumed bytes |
-       * ------------------------------------------------------
-       * 0            start of          end of              buffer
-       *              separator         separator           size
-       *              in buffer         in buffer
-       * }</pre>
-       */
-      private void findSeparatorBounds() throws IOException {
-        int bytePositionInBuffer = 0;
-        while (true) {
-          if (!tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + 1)) {
-            startOfSeparatorInBuffer = endOfSeparatorInBuffer = bytePositionInBuffer;
-            break;
-          }
-
-          byte currentByte = buffer.byteAt(bytePositionInBuffer);
-
-          if (currentByte == '\n') {
-            startOfSeparatorInBuffer = bytePositionInBuffer;
-            endOfSeparatorInBuffer = startOfSeparatorInBuffer + 1;
-            break;
-          } else if (currentByte == '\r') {
-            startOfSeparatorInBuffer = bytePositionInBuffer;
-            endOfSeparatorInBuffer = startOfSeparatorInBuffer + 1;
-
-            if (tryToEnsureNumberOfBytesInBuffer(bytePositionInBuffer + 2)) {
-              currentByte = buffer.byteAt(bytePositionInBuffer + 1);
-              if (currentByte == '\n') {
-                endOfSeparatorInBuffer += 1;
-              }
-            }
-            break;
-          }
-
-          // Move to the next byte in buffer.
-          bytePositionInBuffer += 1;
-        }
-      }
-
-      @Override
-      protected boolean readNextRecord() throws IOException {
-        startOfNextRecord += endOfSeparatorInBuffer;
-        findSeparatorBounds();
-
-        // If we have reached EOF file and consumed all of the buffer then we know
-        // that there are no more records.
-        if (eof && buffer.size() == 0) {
-          elementIsPresent = false;
-          return false;
-        }
-
-        decodeCurrentElement();
-        return true;
-      }
-
-      /**
-       * Decodes the current element updating the buffer to only contain the unconsumed bytes.
-       *
-       * This invalidates the currently stored {@code startOfSeparatorInBuffer} and
-       * {@code endOfSeparatorInBuffer}.
-       */
-      private void decodeCurrentElement() throws IOException {
-        ByteString dataToDecode = buffer.substring(0, startOfSeparatorInBuffer);
-        currentValue = coder.decode(dataToDecode.newInput(), Context.OUTER);
-        elementIsPresent = true;
-        buffer = buffer.substring(endOfSeparatorInBuffer);
-      }
-
-      /**
-       * Returns false if we were unable to ensure the minimum capacity by consuming the channel.
-       */
-      private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOException {
-        // While we aren't at EOF or haven't fulfilled the minimum buffer capacity,
-        // attempt to read more bytes.
-        while (buffer.size() <= minCapacity && !eof) {
-          eof = inChannel.read(readBuffer) == -1;
-          readBuffer.flip();
-          buffer = buffer.concat(ByteString.copyFrom(readBuffer));
-          readBuffer.clear();
-        }
-        // Return true if we were able to honor the minimum buffer capacity request
-        return buffer.size() >= minCapacity;
-      }
-    }
-  }
-
-  /**
-   * A {@link FileBasedSink} for text files. Produces text files with the new line separator
-   * {@code '\n'} represented in {@code UTF-8} format as the record separator.
-   * Each record (including the last) is terminated.
-   */
-  @VisibleForTesting
-  static class TextSink<T> extends FileBasedSink<T> {
-    private final Coder<T> coder;
-
-    @VisibleForTesting
-    TextSink(
-        String baseOutputFilename, String extension, String fileNameTemplate, Coder<T> coder) {
-      super(baseOutputFilename, extension, fileNameTemplate);
-      this.coder = coder;
-    }
-
-    @Override
-    public FileBasedSink.FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options) {
-      return new TextWriteOperation<>(this, coder);
-    }
-
-    /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation
-     * FileBasedWriteOperation} for text files.
-     */
-    private static class TextWriteOperation<T> extends FileBasedWriteOperation<T> {
-      private final Coder<T> coder;
-
-      private TextWriteOperation(TextSink<T> sink, Coder<T> coder) {
-        super(sink);
-        this.coder = coder;
-      }
-
-      @Override
-      public FileBasedWriter<T> createWriter(PipelineOptions options) throws Exception {
-        return new TextWriter<>(this, coder);
-      }
-    }
-
-    /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
-     * for text files.
-     */
-    private static class TextWriter<T> extends FileBasedWriter<T> {
-      private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
-      private final Coder<T> coder;
-      private OutputStream out;
-
-      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder) {
-        super(writeOperation);
-        this.mimeType = MimeTypes.TEXT;
-        this.coder = coder;
-      }
-
-      @Override
-      protected void prepareWrite(WritableByteChannel channel) throws Exception {
-        out = Channels.newOutputStream(channel);
-      }
-
-      @Override
-      public void write(T value) throws Exception {
-        coder.encode(value, out, Context.OUTER);
-        out.write(NEWLINE);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java
deleted file mode 100644
index 9f4a234..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java
+++ /dev/null
@@ -1,254 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Source} that reads an unbounded amount of input and, because of that, supports
- * some additional operations such as checkpointing, watermarks, and record ids.
- *
- * <ul>
- * <li> Checkpointing allows sources to not re-read the same data again in the case of failures.
- * <li> Watermarks allow for downstream parts of the pipeline to know up to what point
- *   in time the data is complete.
- * <li> Record ids allow for efficient deduplication of input records; many streaming sources
- *   do not guarantee that a given record will only be read a single time.
- * </ul>
- *
- * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} and
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for more information on
- * timestamps and watermarks.
- *
- * @param <OutputT> Type of records output by this source.
- * @param <CheckpointMarkT> Type of checkpoint marks used by the readers of this source.
- */
-public abstract class UnboundedSource<
-        OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark> extends Source<OutputT> {
-  /**
-   * Returns a list of {@code UnboundedSource} objects representing the instances of this source
-   * that should be used when executing the workflow.  Each split should return a separate partition
-   * of the input data.
-   *
-   * <p>For example, for a source reading from a growing directory of files, each split
-   * could correspond to a prefix of file names.
-   *
-   * <p>Some sources are not splittable, such as reading from a single TCP stream.  In that
-   * case, only a single split should be returned.
-   *
-   * <p>Some data sources automatically partition their data among readers.  For these types of
-   * inputs, {@code n} identical replicas of the top-level source can be returned.
-   *
-   * <p>The size of the returned list should be as close to {@code desiredNumSplits}
-   * as possible, but does not have to match exactly.  A low number of splits
-   * will limit the amount of parallelism in the source.
-   */
-  public abstract List<? extends UnboundedSource<OutputT, CheckpointMarkT>> generateInitialSplits(
-      int desiredNumSplits, PipelineOptions options) throws Exception;
-
-  /**
-   * Create a new {@link UnboundedReader} to read from this source, resuming from the given
-   * checkpoint if present.
-   */
-  public abstract UnboundedReader<OutputT> createReader(
-      PipelineOptions options, @Nullable CheckpointMarkT checkpointMark);
-
-  /**
-   * Returns a {@link Coder} for encoding and decoding the checkpoints for this source, or
-   * null if the checkpoints do not need to be durably committed.
-   */
-  @Nullable
-  public abstract Coder<CheckpointMarkT> getCheckpointMarkCoder();
-
-  /**
-   * Returns whether this source requires explicit deduping.
-   *
-   * <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.
-   */
-  public boolean requiresDeduping() {
-    return false;
-  }
-
-  /**
-   * A marker representing the progress and state of an
-   * {@link com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader}.
-   *
-   * <p>For example, this could be offsets in a set of files being read.
-   */
-  public interface CheckpointMark {
-    /**
-     * Perform any finalization that needs to happen after a bundle of data read from
-     * the source has been processed and committed.
-     *
-     * <p>For example, this could be sending acknowledgement requests to an external
-     * data source such as Pub/Sub.
-     *
-     * <p>This may be called from any thread, potentially at the same time as calls to the
-     * {@code UnboundedReader} that created it.
-     */
-    void finalizeCheckpoint() throws IOException;
-  }
-
-  /**
-   * A {@code Reader} that reads an unbounded amount of input.
-   *
-   * <p>A given {@code UnboundedReader} object will only be accessed by a single thread at once.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  public abstract static class UnboundedReader<OutputT> extends Source.Reader<OutputT> {
-    private static final byte[] EMPTY = new byte[0];
-
-    /**
-     * Initializes the reader and advances the reader to the first record.
-     *
-     * <p>This method should be called exactly once. The invocation should occur prior to calling
-     * {@link #advance} or {@link #getCurrent}. This method may perform expensive operations that
-     * are needed to initialize the reader.
-     *
-     * <p>Returns {@code true} if a record was read, {@code false} if there is no more input
-     * currently available.  Future calls to {@link #advance} may return {@code true} once more data
-     * is available. Regardless of the return value of {@code start}, {@code start} will not be
-     * called again on the same {@code UnboundedReader} object; it will only be called again when a
-     * new reader object is constructed for the same source, e.g. on recovery.
-     */
-    @Override
-    public abstract boolean start() throws IOException;
-
-    /**
-     * Advances the reader to the next valid record.
-     *
-     * <p>Returns {@code true} if a record was read, {@code false} if there is no more input
-     * available. Future calls to {@link #advance} may return {@code true} once more data is
-     * available.
-     */
-    @Override
-    public abstract boolean advance() throws IOException;
-
-    /**
-     * Returns a unique identifier for the current record.  This should be the same for each
-     * instance of the same logical record read from the underlying data source.
-     *
-     * <p>It is only necessary to override this if {@link #requiresDeduping} has been overridden to
-     * return true.
-     *
-     * <p>For example, this could be a hash of the record contents, or a logical ID present in
-     * the record.  If this is generated as a hash of the record contents, it should be at least 16
-     * bytes (128 bits) to avoid collisions.
-     *
-     * <p>This method has the same restrictions on when it can be called as {@link #getCurrent} and
-     * {@link #getCurrentTimestamp}.
-     *
-     * @throws NoSuchElementException if the reader is at the beginning of the input and
-     *         {@link #start} or {@link #advance} wasn't called, or if the last {@link #start} or
-     *         {@link #advance} returned {@code false}.
-     */
-    public byte[] getCurrentRecordId() throws NoSuchElementException {
-      if (getCurrentSource().requiresDeduping()) {
-        throw new IllegalStateException(
-            "getCurrentRecordId() must be overridden if requiresDeduping returns true()");
-      }
-      return EMPTY;
-    }
-
-    /**
-     * Returns a timestamp before or at the timestamps of all future elements read by this reader.
-     *
-     * <p>This can be approximate.  If records are read that violate this guarantee, they will be
-     * considered late, which will affect how they will be processed.  See
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} for more information on
-     * late data and how to handle it.
-     *
-     * <p>However, this value should be as late as possible. Downstream windows may not be able
-     * to close until this watermark passes their end.
-     *
-     * <p>For example, a source may know that the records it reads will be in timestamp order.  In
-     * this case, the watermark can be the timestamp of the last record read.  For a
-     * source that does not have natural timestamps, timestamps can be set to the time of
-     * reading, in which case the watermark is the current clock time.
-     *
-     * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} and
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for more
-     * information on timestamps and watermarks.
-     *
-     * <p>May be called after {@link #advance} or {@link #start} has returned false, but not before
-     * {@link #start} has been called.
-     */
-    public abstract Instant getWatermark();
-
-    /**
-     * Returns a {@link CheckpointMark} representing the progress of this {@code UnboundedReader}.
-     *
-     * <p>The elements read up until this is called will be processed together as a bundle. Once
-     * the result of this processing has been durably committed,
-     * {@link CheckpointMark#finalizeCheckpoint} will be called on the {@link CheckpointMark}
-     * object.
-     *
-     * <p>The returned object should not be modified.
-     *
-     * <p>May be called after {@link #advance} or {@link #start} has returned false, but not before
-     * {@link #start} has been called.
-     */
-    public abstract CheckpointMark getCheckpointMark();
-
-    /**
-     * Constant representing an unknown amount of backlog.
-     */
-    public static final long BACKLOG_UNKNOWN = -1L;
-
-    /**
-     * Returns the size of the backlog of unread data in the underlying data source represented by
-     * this split of this source.
-     *
-     * <p>One of this or {@link #getTotalBacklogBytes} should be overridden in order to allow the
-     * runner to scale the amount of resources allocated to the pipeline.
-     */
-    public long getSplitBacklogBytes() {
-      return BACKLOG_UNKNOWN;
-    }
-
-    /**
-     * Returns the size of the backlog of unread data in the underlying data source represented by
-     * all splits of this source.
-     *
-     * <p>One of this or {@link #getSplitBacklogBytes} should be overridden in order to allow the
-     * runner to scale the amount of resources allocated to the pipeline.
-     */
-    public long getTotalBacklogBytes() {
-      return BACKLOG_UNKNOWN;
-    }
-
-    /**
-     * Returns the {@link UnboundedSource} that created this reader.  This will not change over the
-     * life of the reader.
-     */
-    @Override
-    public abstract UnboundedSource<OutputT, ?> getCurrentSource();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java
deleted file mode 100644
index a70105f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java
+++ /dev/null
@@ -1,217 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-import org.joda.time.Instant;
-
-import java.util.UUID;
-
-/**
- * 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.
- *
- * <p>Currently, only batch workflows can contain Write transforms.
- *
- * <p>Example usage:
- *
- * <p>{@code p.apply(Write.to(new MySink(...)));}
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public class Write {
-  /**
-   * Creates a Write transform that writes to the given Sink.
-   */
-  public static <T> Bound<T> to(Sink<T> sink) {
-    return new Bound<>(sink);
-  }
-
-  /**
-   * A {@link PTransform} that writes to a {@link Sink}. See {@link Write} and {@link Sink} for
-   * documentation about writing to Sinks.
-   */
-  public static class Bound<T> extends PTransform<PCollection<T>, PDone> {
-    private final Sink<T> sink;
-
-    private Bound(Sink<T> sink) {
-      this.sink = sink;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      PipelineOptions options = input.getPipeline().getOptions();
-      sink.validate(options);
-      return createWrite(input, sink.createWriteOperation(options));
-    }
-
-    /**
-     * Returns the {@link Sink} associated with this PTransform.
-     */
-    public Sink<T> getSink() {
-      return sink;
-    }
-
-    /**
-     * 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
-     * {@link WriteOperation}. In this initialization ParDo, {@link WriteOperation#initialize} is
-     * called. The output of this ParDo is a singleton PCollection
-     * containing the WriteOperation.
-     *
-     * <p>This singleton collection containing the WriteOperation is then used as a side input to a
-     * ParDo over the PCollection of elements to write. In this bundle-writing phase,
-     * {@link WriteOperation#createWriter} is called to obtain a {@link Writer}.
-     * {@link Writer#open} and {@link Writer#close} are called in {@link DoFn#startBundle} and
-     * {@link DoFn#finishBundle}, respectively, and {@link Writer#write} method is called for every
-     * element in the bundle. The output of this ParDo is a PCollection of <i>writer result</i>
-     * objects (see {@link Sink} for a description of writer results)-one for each bundle.
-     *
-     * <p>The final do-once ParDo uses the singleton collection of the WriteOperation as input and
-     * the collection of writer results as a side-input. In this ParDo,
-     * {@link WriteOperation#finalize} is called to finalize the write.
-     *
-     * <p>If the write of any element in the PCollection fails, {@link Writer#close} will be called
-     * before the exception that caused the write to fail is propagated and the write result will be
-     * discarded.
-     *
-     * <p>Since the {@link WriteOperation} is serialized after the initialization ParDo and
-     * deserialized in the bundle-writing and finalization phases, any state change to the
-     * WriteOperation object that occurs during initialization is visible in the latter phases.
-     * However, the WriteOperation is not serialized after the bundle-writing phase.  This is why
-     * implementations should guarantee that {@link WriteOperation#createWriter} does not mutate
-     * WriteOperation).
-     */
-    private <WriteT> PDone createWrite(
-        PCollection<T> input, WriteOperation<T, WriteT> writeOperation) {
-      Pipeline p = input.getPipeline();
-
-      // A coder to use for the WriteOperation.
-      @SuppressWarnings("unchecked")
-      Coder<WriteOperation<T, WriteT>> operationCoder =
-          (Coder<WriteOperation<T, WriteT>>) SerializableCoder.of(writeOperation.getClass());
-
-      // 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));
-
-      // Initialize the resource in a do-once ParDo on the WriteOperation.
-      operationCollection = operationCollection
-          .apply("Initialize", ParDo.of(
-              new DoFn<WriteOperation<T, WriteT>, WriteOperation<T, WriteT>>() {
-            @Override
-            public void processElement(ProcessContext c) throws Exception {
-              WriteOperation<T, WriteT> writeOperation = c.element();
-              writeOperation.initialize(c.getPipelineOptions());
-              // The WriteOperation is also the output of this ParDo, so it can have mutable
-              // state.
-              c.output(writeOperation);
-            }
-          }))
-          .setCoder(operationCoder);
-
-      // Create a view of the WriteOperation to be used as a sideInput to the parallel write phase.
-      final PCollectionView<WriteOperation<T, WriteT>> writeOperationView =
-          operationCollection.apply(View.<WriteOperation<T, WriteT>>asSingleton());
-
-      // 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("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);
-                writer = writeOperation.createWriter(c.getPipelineOptions());
-                writer.open(UUID.randomUUID().toString());
-              }
-              try {
-                writer.write(c.element());
-              } catch (Exception e) {
-                // Discard write result and close the write.
-                try {
-                  writer.close();
-                } 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();
-                // Output the result of the write.
-                c.outputWithTimestamp(result, Instant.now());
-              }
-            }
-          }).withSideInputs(writeOperationView))
-          .setCoder(writeOperation.getWriterResultCoder())
-          .apply(Window.<WriteT>into(new GlobalWindows()));
-
-      final PCollectionView<Iterable<WriteT>> resultsView =
-          results.apply(View.<WriteT>asIterable());
-
-      // Finalize the write in another do-once ParDo on the singleton collection containing the
-      // Writer. The results from the per-bundle writes are given as an Iterable side input.
-      // The WriteOperation's state is the same as after its initialization in the first do-once
-      // ParDo. There is a dependency between this ParDo and the parallel write (the writer results
-      // collection as a side input), so it will happen after the parallel write.
-      @SuppressWarnings("unused")
-      final PCollection<Integer> done = operationCollection
-          .apply("Finalize", ParDo.of(new DoFn<WriteOperation<T, WriteT>, Integer>() {
-            @Override
-            public void processElement(ProcessContext c) throws Exception {
-              Iterable<WriteT> results = c.sideInput(resultsView);
-              WriteOperation<T, WriteT> writeOperation = c.element();
-              writeOperation.finalize(results, c.getPipelineOptions());
-            }
-          }).withSideInputs(resultsView));
-      return PDone.in(input.getPipeline());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java
deleted file mode 100644
index 9ba76c5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java
+++ /dev/null
@@ -1,311 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Preconditions;
-
-import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-
-// CHECKSTYLE.OFF: JavadocStyle
-/**
- * A {@link Sink} that outputs records as XML-formatted elements. Writes a {@link PCollection} of
- * records from JAXB-annotated classes to a single file location.
- *
- * <p>Given a PCollection containing records of type T that can be marshalled to XML elements, this
- * Sink will produce a single file consisting of a single root element that contains all of the
- * elements in the PCollection.
- *
- * <p>XML Sinks are created with a base filename to write to, a root element name that will be used
- * for the root element of the output files, and a class to bind to an XML element. This class
- * will be used in the marshalling of records in an input PCollection to their XML representation
- * and must be able to be bound using JAXB annotations (checked at pipeline construction time).
- *
- * <p>XML Sinks can be written to using the {@link Write} transform:
- *
- * <pre>
- * p.apply(Write.to(
- *      XmlSink.ofRecordClass(Type.class)
- *          .withRootElementName(root_element)
- *          .toFilenamePrefix(output_filename)));
- * </pre>
- *
- * <p>For example, consider the following class with JAXB annotations:
- *
- * <pre>
- *  {@literal @}XmlRootElement(name = "word_count_result")
- *  {@literal @}XmlType(propOrder = {"word", "frequency"})
- *  public class WordFrequency {
- *    private String word;
- *    private long frequency;
- *
- *    public WordFrequency() { }
- *
- *    public WordFrequency(String word, long frequency) {
- *      this.word = word;
- *      this.frequency = frequency;
- *    }
- *
- *    public void setWord(String word) {
- *      this.word = word;
- *    }
- *
- *    public void setFrequency(long frequency) {
- *      this.frequency = frequency;
- *    }
- *
- *    public long getFrequency() {
- *      return frequency;
- *    }
- *
- *    public String getWord() {
- *      return word;
- *    }
- *  }
- * </pre>
- *
- * <p>The following will produce XML output with a root element named "words" from a PCollection of
- * WordFrequency objects:
- * <pre>
- * p.apply(Write.to(
- *  XmlSink.ofRecordClass(WordFrequency.class)
- *      .withRootElement("words")
- *      .toFilenamePrefix(output_file)));
- * </pre>
- *
- * <p>The output of which will look like:
- * <pre>
- * {@code
- * <words>
- *
- *  <word_count_result>
- *    <word>decreased</word>
- *    <frequency>1</frequency>
- *  </word_count_result>
- *
- *  <word_count_result>
- *    <word>War</word>
- *    <frequency>4</frequency>
- *  </word_count_result>
- *
- *  <word_count_result>
- *    <word>empress'</word>
- *    <frequency>14</frequency>
- *  </word_count_result>
- *
- *  <word_count_result>
- *    <word>stoops</word>
- *    <frequency>6</frequency>
- *  </word_count_result>
- *
- *  ...
- * </words>
- * }</pre>
- */
-// CHECKSTYLE.ON: JavadocStyle
-@SuppressWarnings("checkstyle:javadocstyle")
-public class XmlSink {
-  protected static final String XML_EXTENSION = "xml";
-
-  /**
-   * Returns a builder for an XmlSink. You'll need to configure the class to bind, the root
-   * element name, and the output file prefix with {@link Bound#ofRecordClass}, {@link
-   * Bound#withRootElement}, and {@link Bound#toFilenamePrefix}, respectively.
-   */
-  public static Bound<?> write() {
-    return new Bound<>(null, null, null);
-  }
-
-  /**
-   * Returns an XmlSink that writes objects as XML entities.
-   *
-   * <p>Output files will have the name {@literal {baseOutputFilename}-0000i-of-0000n.xml} where n
-   * is the number of output bundles that the Dataflow service divides the output into.
-   *
-   * @param klass the class of the elements to write.
-   * @param rootElementName the enclosing root element.
-   * @param baseOutputFilename the output filename prefix.
-   */
-  public static <T> Bound<T> writeOf(
-      Class<T> klass, String rootElementName, String baseOutputFilename) {
-    return new Bound<>(klass, rootElementName, baseOutputFilename);
-  }
-
-  /**
-   * A {@link FileBasedSink} that writes objects as XML elements.
-   */
-  public static class Bound<T> extends FileBasedSink<T> {
-    final Class<T> classToBind;
-    final String rootElementName;
-
-    private Bound(Class<T> classToBind, String rootElementName, String baseOutputFilename) {
-      super(baseOutputFilename, XML_EXTENSION);
-      this.classToBind = classToBind;
-      this.rootElementName = rootElementName;
-    }
-
-    /**
-     * Returns an XmlSink that writes objects of the class specified as XML elements.
-     *
-     * <p>The specified class must be able to be used to create a JAXB context.
-     */
-    public <T> Bound<T> ofRecordClass(Class<T> classToBind) {
-      return new Bound<>(classToBind, rootElementName, baseOutputFilename);
-    }
-
-    /**
-     * Returns an XmlSink that writes to files with the given prefix.
-     *
-     * <p>Output files will have the name {@literal {filenamePrefix}-0000i-of-0000n.xml} where n is
-     * the number of output bundles that the Dataflow service divides the output into.
-     */
-    public Bound<T> toFilenamePrefix(String baseOutputFilename) {
-      return new Bound<>(classToBind, rootElementName, baseOutputFilename);
-    }
-
-    /**
-     * Returns an XmlSink that writes XML files with an enclosing root element of the
-     * supplied name.
-     */
-    public Bound<T> withRootElement(String rootElementName) {
-      return new Bound<>(classToBind, rootElementName, baseOutputFilename);
-    }
-
-    /**
-     * Validates that the root element, class to bind to a JAXB context, and filenamePrefix have
-     * been set and that the class can be bound in a JAXB context.
-     */
-    @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.");
-      try {
-        JAXBContext.newInstance(classToBind);
-      } catch (JAXBException e) {
-        throw new RuntimeException("Error binding classes to a JAXB Context.", e);
-      }
-    }
-
-    /**
-     * Creates an {@link XmlWriteOperation}.
-     */
-    @Override
-    public XmlWriteOperation<T> createWriteOperation(PipelineOptions options) {
-      return new XmlWriteOperation<>(this);
-    }
-  }
-
-  /**
-   * {@link Sink.WriteOperation} for XML {@link Sink}s.
-   */
-  protected static final class XmlWriteOperation<T> extends FileBasedWriteOperation<T> {
-    public XmlWriteOperation(XmlSink.Bound<T> sink) {
-      super(sink);
-    }
-
-    /**
-     * Creates a {@link XmlWriter} with a marshaller for the type it will write.
-     */
-    @Override
-    public XmlWriter<T> createWriter(PipelineOptions options) throws Exception {
-      JAXBContext context;
-      Marshaller marshaller;
-      context = JAXBContext.newInstance(getSink().classToBind);
-      marshaller = context.createMarshaller();
-      marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
-      marshaller.setProperty(Marshaller.JAXB_FRAGMENT, Boolean.TRUE);
-      marshaller.setProperty(Marshaller.JAXB_ENCODING, "UTF-8");
-      return new XmlWriter<>(this, marshaller);
-    }
-
-    /**
-     * Return the XmlSink.Bound for this write operation.
-     */
-    @Override
-    public XmlSink.Bound<T> getSink() {
-      return (XmlSink.Bound<T>) super.getSink();
-    }
-  }
-
-  /**
-   * A {@link Sink.Writer} that can write objects as XML elements.
-   */
-  protected static final class XmlWriter<T> extends FileBasedWriter<T> {
-    final Marshaller marshaller;
-    private OutputStream os = null;
-
-    public XmlWriter(XmlWriteOperation<T> writeOperation, Marshaller marshaller) {
-      super(writeOperation);
-      this.marshaller = marshaller;
-    }
-
-    /**
-     * Creates the output stream that elements will be written to.
-     */
-    @Override
-    protected void prepareWrite(WritableByteChannel channel) throws Exception {
-      os = Channels.newOutputStream(channel);
-    }
-
-    /**
-     * Writes the root element opening tag.
-     */
-    @Override
-    protected void writeHeader() throws Exception {
-      String rootElementName = getWriteOperation().getSink().rootElementName;
-      os.write(CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "<" + rootElementName + ">\n"));
-    }
-
-    /**
-     * Writes the root element closing tag.
-     */
-    @Override
-    protected void writeFooter() throws Exception {
-      String rootElementName = getWriteOperation().getSink().rootElementName;
-      os.write(CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "\n</" + rootElementName + ">"));
-    }
-
-    /**
-     * Writes a value to the stream.
-     */
-    @Override
-    public void write(T value) throws Exception {
-      marshaller.marshal(value, os);
-    }
-
-    /**
-     * Return the XmlWriteOperation this write belongs to.
-     */
-    @Override
-    public XmlWriteOperation<T> getWriteOperation() {
-      return (XmlWriteOperation<T>) super.getWriteOperation();
-    }
-  }
-}


[67/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
index 18e6654..8fcfccf 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import static com.google.cloud.dataflow.sdk.util.Transport.getJsonFactory;
-import static com.google.cloud.dataflow.sdk.util.Transport.getTransport;
+import static org.apache.beam.sdk.util.Transport.getJsonFactory;
+import static org.apache.beam.sdk.util.Transport.getTransport;
+
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.http.HttpRequestInitializer;
 import com.google.api.services.clouddebugger.v2.Clouddebugger;
 import com.google.api.services.dataflow.Dataflow;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
 import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
index 7307e83..4f1f673 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 import com.google.common.base.Preconditions;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
index 2c06a92..5afca52 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
+
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
 
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
 import com.google.api.services.dataflow.model.JobMessage;
 import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableMap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
index 0e234a8..7d492b4 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.util.BackOffUtils;
 import com.google.api.client.util.Sleeper;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
index f6c6a71..9547ac1 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.services.dataflow.model.DataflowPackage;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
index 7788b5b..6b9fbb4 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.testing.TestDataflowPipelineOptions;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
index 1b5a3c7..c3f3a18 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.hamcrest.Matchers.hasEntry;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
index eff79bb..c9eac56 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.testing.ResetDateTimeProvider;
-import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties;
+import org.apache.beam.sdk.testing.ResetDateTimeProvider;
+import org.apache.beam.sdk.testing.RestoreSystemProperties;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
index 1420273..18c8085 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
index b752f3d..47d518d 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
@@ -15,12 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import static org.apache.beam.sdk.options.DataflowWorkerLoggingOptions.Level.WARN;
 
-import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.WARN;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
+import org.apache.beam.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
+
 import com.google.common.collect.ImmutableMap;
 
 import com.fasterxml.jackson.databind.ObjectMapper;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
index 0322426..67ecdbe 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
@@ -24,16 +24,16 @@ import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+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.testing.TestDataflowPipelineOptions;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
 
 import org.hamcrest.Description;
 import org.hamcrest.Factory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
index 764c0cb..d496f38 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
@@ -32,6 +32,19 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
 import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics;
@@ -40,18 +53,6 @@ import com.google.api.services.dataflow.model.Job;
 import com.google.api.services.dataflow.model.JobMetrics;
 import com.google.api.services.dataflow.model.MetricStructuredName;
 import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSetMultimap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
index 4850939..67ed2fd 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
index c6dca2d..303acda 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
 
-import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInGlobalWindow;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.instanceOf;
@@ -35,59 +36,60 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+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.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsList;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsMap;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsMultimap;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner.TransformedMap;
+import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
+import org.apache.beam.sdk.runners.worker.IsmFormat;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.DataflowReleaseInfo;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+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.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.model.DataflowPackage;
 import com.google.api.services.dataflow.model.Job;
 import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.AvroSource;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsList;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMap;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMultimap;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.TransformedMap;
-import com.google.cloud.dataflow.sdk.runners.dataflow.TestCountingSource;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
index ebf0c9f..947b599 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
index dd1b3c8..97d1a50 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import static org.apache.beam.sdk.util.Structs.addObject;
+import static org.apache.beam.sdk.util.Structs.getDictionary;
+import static org.apache.beam.sdk.util.Structs.getString;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
-import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.getString;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.hasKey;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
@@ -34,40 +35,41 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineWorkerPoolOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+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.Sum;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.OutputReference;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.model.DataflowPackage;
 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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.OutputReference;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
@@ -188,8 +190,8 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Map<String, Object> settings = new HashMap<>();
     settings.put("appName", "DataflowPipelineTranslatorTest");
     settings.put("project", "some-project");
-    settings.put("pathValidatorClass", "com.google.cloud.dataflow.sdk.util.DataflowPathValidator");
-    settings.put("runner", "com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner");
+    settings.put("pathValidatorClass", "org.apache.beam.sdk.util.DataflowPathValidator");
+    settings.put("runner", "org.apache.beam.sdk.runners.DataflowPipelineRunner");
     settings.put("jobName", "some-job-name");
     settings.put("tempLocation", "gs://somebucket/some/path");
     settings.put("stagingLocation", "gs://somebucket/some/path/staging");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
index 20d2bc8..2acede3 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
+package org.apache.beam.sdk.runners.dataflow;
+import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource;
+
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.contains;
 import static org.junit.Assert.assertEquals;
@@ -25,23 +26,24 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Sample;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+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.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Sample;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Preconditions;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
index 0a78a6d..b480c0b 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
@@ -30,6 +30,21 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineJob;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.MonitoringUtil.JobMessagesHandler;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.TimeUtil;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.api.client.http.LowLevelHttpResponse;
 import com.google.api.client.json.Json;
 import com.google.api.client.testing.http.MockHttpTransport;
@@ -40,20 +55,6 @@ import com.google.api.services.dataflow.model.JobMessage;
 import com.google.api.services.dataflow.model.JobMetrics;
 import com.google.api.services.dataflow.model.MetricStructuredName;
 import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.TimeUtil;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
index 914a484..a381f68 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
@@ -15,23 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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.TypeDescriptor;
 
 import org.joda.time.Duration;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
index 936b7c6..b86de7e 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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.PCollectionView;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
index 7f35fd3..b459c47 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
index 23e12de..bdc0bc3 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.model.JobMessage;
 import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
 
 import org.joda.time.Instant;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
index d2c08c8..21bc60e 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
@@ -32,6 +32,13 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.util.PackageUtil.PackageAttributes;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
 import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.http.HttpRequest;
@@ -48,12 +55,6 @@ import com.google.api.client.testing.http.MockHttpTransport;
 import com.google.api.client.testing.http.MockLowLevelHttpRequest;
 import com.google.api.client.testing.http.MockLowLevelHttpResponse;
 import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
-import com.google.cloud.dataflow.sdk.util.PackageUtil.PackageAttributes;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 411b4cc..7b2f356 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -53,7 +53,7 @@
                 <goal>test</goal>
               </goals>
               <configuration>
-                <groups>com.google.cloud.dataflow.sdk.testing.RunnableOnService</groups>
+                <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
                 <parallel>all</parallel>
                 <threadCount>4</threadCount>
                 <dependenciesToScan>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
index fa4776d..5fd6bd4 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java
@@ -18,9 +18,9 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
 
 /**
  * Interface for retrieving the result(s) of running a pipeline. Allows us to translate between

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
index a7f65cc..bdf832b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
@@ -18,11 +18,11 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
 
 public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions,
                                               ApplicationNameOptions {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
index 71e358c..99da74f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -18,24 +18,28 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import org.apache.beam.runners.spark.translation.*;
+import org.apache.beam.runners.spark.translation.EvaluationContext;
+import org.apache.beam.runners.spark.translation.SparkContextFactory;
+import org.apache.beam.runners.spark.translation.SparkPipelineEvaluator;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
+import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
+import org.apache.beam.runners.spark.translation.SparkProcessContext;
+import org.apache.beam.runners.spark.translation.TransformTranslator;
 import org.apache.beam.runners.spark.translation.streaming.StreamingEvaluationContext;
 import org.apache.beam.runners.spark.translation.streaming.StreamingTransformTranslator;
 import org.apache.beam.runners.spark.translation.streaming.StreamingWindowPipelineDetector;
-
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
 import org.apache.spark.SparkException;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.streaming.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
index be40313..091d0de 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkStreamingPipelineOptions.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
 
 /**
  * Options used to configure Spark streaming.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
index 6c397a2..64c473e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
@@ -25,12 +25,11 @@ import java.io.Serializable;
 import java.util.Map;
 import java.util.TreeMap;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
 import com.google.common.collect.ImmutableList;
-
 import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine;
 
 /**
  * This class wraps a map of named aggregators. Spark expects that all accumulators be declared

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
index 11ce447..7dc6af6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
@@ -24,9 +24,9 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
 import com.google.common.collect.Iterables;
 import org.apache.beam.runners.spark.util.ByteArray;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.PairFunction;
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
index 08a38e7..529d67b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java
@@ -22,7 +22,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.hadoop.io.NullWritable;
 
 public final class NullWritableCoder extends WritableCoder<NullWritable> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
index 1378977..f2836fe 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java
@@ -28,10 +28,10 @@ import java.util.List;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
+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.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
index 681dc72..41dc367 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.runners.spark.io;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
 
 /**
  * Print to console.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7033104..917f8a0 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,11 +17,11 @@
  */
 package org.apache.beam.runners.spark.io;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.common.base.Preconditions;
+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;
 
 /**
  * Create an input stream from Queue.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0714d38..1592bec 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
@@ -20,14 +20,13 @@ package org.apache.beam.runners.spark.io;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.common.base.Preconditions;
-
 import kafka.serializer.Decoder;
+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;
 
 /**
  * Read stream from Kafka.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b9065fa..5b50d3e 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
@@ -20,14 +20,14 @@ package org.apache.beam.runners.spark.io.hadoop;
 import java.util.HashMap;
 import java.util.Map;
 
-import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.io.ShardNameTemplate;
+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.PDone;
+import org.apache.beam.sdk.values.PInput;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
index 9e3ba82..c768340 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java
@@ -31,7 +31,7 @@ public final class ShardNameBuilder {
   /**
    * Replace occurrences of uppercase letters 'N' with the given {code}shardCount{code},
    * left-padded with zeros if necessary.
-   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
+   * @see org.apache.beam.sdk.io.ShardNameTemplate
    * @param template the string template containing uppercase letters 'N'
    * @param shardCount the total number of shards
    * @return a string template with 'N' replaced by the shard count
@@ -43,7 +43,7 @@ public final class ShardNameBuilder {
   /**
    * Replace occurrences of uppercase letters 'S' with the given {code}shardNumber{code},
    * left-padded with zeros if necessary.
-   * @see com.google.cloud.dataflow.sdk.io.ShardNameTemplate
+   * @see org.apache.beam.sdk.io.ShardNameTemplate
    * @param template the string template containing uppercase letters 'S'
    * @param shardNumber the number of a particular shard
    * @return a string template with 'S' replaced by the shard number

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
index a4fd6f0..0a30f9f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateAware.java
@@ -25,7 +25,7 @@ package org.apache.beam.runners.spark.io.hadoop;
  * {@link HadoopIO.Write}.
  *
  * Some common shard names are defined in
- * {@link com.google.cloud.dataflow.sdk.io.ShardNameTemplate}.
+ * {@link org.apache.beam.sdk.io.ShardNameTemplate}.
  */
 public interface ShardNameTemplateAware {
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
index 44fe26c..2267ccb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java
@@ -18,10 +18,10 @@
 
 package org.apache.beam.runners.spark.io.hadoop;
 
-import java.io.IOException;
-
 import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber;
 
+import java.io.IOException;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskID;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
index 37bf849..674da73 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
@@ -23,10 +23,10 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
 import org.apache.spark.api.java.function.FlatMapFunction;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
index a1ddd44..78a62aa 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
@@ -26,23 +26,23 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
 import org.apache.spark.api.java.JavaRDDLike;
 import org.apache.spark.api.java.JavaSparkContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
index cecf962..2641e31 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
@@ -21,14 +21,14 @@ package org.apache.beam.runners.spark.translation;
 import java.util.Iterator;
 import java.util.Map;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.base.Function;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
 import org.apache.spark.api.java.function.PairFlatMapFunction;
 import org.joda.time.Instant;
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
index 0186c8c..0f47af6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineEvaluator.java
@@ -18,12 +18,12 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 /**
  * Pipeline {@link SparkPipelineRunner.Evaluator} for Spark.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
index 2b6804e..fe89ee3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsFactory.java
@@ -18,8 +18,8 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 public final class SparkPipelineOptionsFactory {
   private SparkPipelineOptionsFactory() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
index 9775b3e..c882d7b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsRegistrar.java
@@ -18,10 +18,10 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 
 public class SparkPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
index e44d999..38993fb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineRunnerRegistrar.java
@@ -18,10 +18,10 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
 public class SparkPipelineRunnerRegistrar implements PipelineRunnerRegistrar {
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
index ac1c685..77849a9 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * Translator to support translation between Dataflow transformations and Spark transformations.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
index bfcdd80..5d4ece6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
@@ -23,23 +23,23 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
-
 import org.apache.beam.runners.spark.util.BroadcastHelper;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;


[33/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
new file mode 100644
index 0000000..ebd42d9
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
@@ -0,0 +1,155 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import com.google.common.base.Preconditions;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Options that are used to control logging configuration on the Dataflow worker.
+ */
+@Description("Options that are used to control logging configuration on the Dataflow worker.")
+public interface DataflowWorkerLoggingOptions extends PipelineOptions {
+  /**
+   * The set of log levels that can be used on the Dataflow worker.
+   */
+  public enum Level {
+    DEBUG, ERROR, INFO, TRACE, WARN
+  }
+
+  /**
+   * This option controls the default log level of all loggers without a log level override.
+   */
+  @Description("Controls the default log level of all loggers without a log level override.")
+  @Default.Enum("INFO")
+  Level getDefaultWorkerLogLevel();
+  void setDefaultWorkerLogLevel(Level level);
+
+  /**
+   * This option controls the log levels for specifically named loggers.
+   *
+   * <p>Later options with equivalent names override earlier options.
+   *
+   * <p>See {@link WorkerLogLevelOverrides} for more information on how to configure logging
+   * on a per {@link Class}, {@link Package}, or name basis. If used from the command line,
+   * the expected format is {"Name":"Level",...}, further details on
+   * {@link WorkerLogLevelOverrides#from}.
+   */
+  @Description("This option controls the log levels for specifically named loggers. "
+      + "The expected format is {\"Name\":\"Level\",...}. The Dataflow worker uses "
+      + "java.util.logging, which supports a logging hierarchy based off of names that are '.' "
+      + "separated. For example, by specifying the value {\"a.b.c.Foo\":\"DEBUG\"}, the logger "
+      + "for the class 'a.b.c.Foo' will be configured to output logs at the DEBUG level. "
+      + "Similarly, by specifying the value {\"a.b.c\":\"WARN\"}, all loggers underneath the "
+      + "'a.b.c' package will be configured to output logs at the WARN level. Also, note that "
+      + "when multiple overrides are specified, the exact name followed by the closest parent "
+      + "takes precedence.")
+  WorkerLogLevelOverrides getWorkerLogLevelOverrides();
+  void setWorkerLogLevelOverrides(WorkerLogLevelOverrides value);
+
+  /**
+   * Defines a log level override for a specific class, package, or name.
+   *
+   * <p>{@code java.util.logging} is used on the Dataflow worker harness and supports
+   * a logging hierarchy based off of names that are "." separated. It is a common
+   * pattern to have the logger for a given class share the same name as the class itself.
+   * Given the classes {@code a.b.c.Foo}, {@code a.b.c.Xyz}, and {@code a.b.Bar}, with
+   * loggers named {@code "a.b.c.Foo"}, {@code "a.b.c.Xyz"}, and {@code "a.b.Bar"} respectively,
+   * we can override the log levels:
+   * <ul>
+   *    <li>for {@code Foo} by specifying the name {@code "a.b.c.Foo"} or the {@link Class}
+   *    representing {@code a.b.c.Foo}.
+   *    <li>for {@code Foo}, {@code Xyz}, and {@code Bar} by specifying the name {@code "a.b"} or
+   *    the {@link Package} representing {@code a.b}.
+   *    <li>for {@code Foo} and {@code Bar} by specifying both of their names or classes.
+   * </ul>
+   * Note that by specifying multiple overrides, the exact name followed by the closest parent
+   * takes precedence.
+   */
+  public static class WorkerLogLevelOverrides extends HashMap<String, Level> {
+    /**
+     * Overrides the default log level for the passed in class.
+     *
+     * <p>This is equivalent to calling
+     * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
+     * 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.");
+      addOverrideForName(klass.getName(), level);
+      return this;
+    }
+
+    /**
+     * Overrides the default log level for the passed in package.
+     *
+     * <p>This is equivalent to calling
+     * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
+     * 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.");
+      addOverrideForName(pkg.getName(), level);
+      return this;
+    }
+
+    /**
+     * Overrides the default log level for the passed in name.
+     *
+     * <p>Note that because of the hierarchical nature of logger names, this will
+     * override the log level of all loggers that have the passed in name or
+     * 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,
+          "Expected level to be one of %s.", Arrays.toString(Level.values()));
+      put(name, level);
+      return this;
+    }
+
+    /**
+     * Expects a map keyed by logger {@code Name}s with values representing {@code Level}s.
+     * The {@code Name} generally represents the fully qualified Java
+     * {@link Class#getName() class name}, or fully qualified Java
+     * {@link Package#getName() package name}, or custom logger name. The {@code Level}
+     * represents the log level and must be one of {@link Level}.
+     */
+    @JsonCreator
+    public static WorkerLogLevelOverrides from(Map<String, String> values) {
+      Preconditions.checkNotNull(values, "Expected values to be not null.");
+      WorkerLogLevelOverrides overrides = new WorkerLogLevelOverrides();
+      for (Map.Entry<String, String> entry : values.entrySet()) {
+        try {
+          overrides.addOverrideForName(entry.getKey(), Level.valueOf(entry.getValue()));
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException(String.format(
+              "Unsupported log level '%s' requested for %s. Must be one of %s.",
+              entry.getValue(), entry.getKey(), Arrays.toString(Level.values())));
+        }
+
+      }
+      return overrides;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
new file mode 100644
index 0000000..9096c2b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
@@ -0,0 +1,186 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link PipelineRunner} that's like {@link DataflowPipelineRunner}
+ * but that waits for the launched job to finish.
+ *
+ * <p>Prints out job status updates and console messages while it waits.
+ *
+ * <p>Returns the final job state, or throws an exception if the job
+ * fails or cannot be monitored.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code BlockingDataflowPipelineRunner}, the Google cloud services account and the Google compute
+ * engine service account of the GCP project running the Dataflow Job will need access to the
+ * corresponding source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class BlockingDataflowPipelineRunner extends
+    PipelineRunner<DataflowPipelineJob> {
+  private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class);
+
+  // Defaults to an infinite wait period.
+  // TODO: make this configurable after removal of option map.
+  private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
+
+  private final DataflowPipelineRunner dataflowPipelineRunner;
+  private final BlockingDataflowPipelineOptions options;
+
+  protected BlockingDataflowPipelineRunner(
+      DataflowPipelineRunner internalRunner,
+      BlockingDataflowPipelineOptions options) {
+    this.dataflowPipelineRunner = internalRunner;
+    this.options = options;
+  }
+
+  /**
+   * Constructs a runner from the provided options.
+   */
+  public static BlockingDataflowPipelineRunner fromOptions(
+      PipelineOptions options) {
+    BlockingDataflowPipelineOptions dataflowOptions =
+        PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
+    DataflowPipelineRunner dataflowPipelineRunner =
+        DataflowPipelineRunner.fromOptions(dataflowOptions);
+
+    return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, dataflowOptions);
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws DataflowJobExecutionException if there is an exception during job execution.
+   * @throws DataflowServiceException if there is an exception retrieving information about the job.
+   */
+  @Override
+  public DataflowPipelineJob run(Pipeline p) {
+    final DataflowPipelineJob job = dataflowPipelineRunner.run(p);
+
+    // We ignore the potential race condition here (Ctrl-C after job submission but before the
+    // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
+    // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
+    // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
+    // etc. If the user wants to verify the job was cancelled they should look at the job status.
+    Thread shutdownHook = new Thread() {
+      @Override
+      public void run() {
+        LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
+            + "To cancel the job in the cloud, run:\n> {}",
+            MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
+      }
+    };
+
+    try {
+      Runtime.getRuntime().addShutdownHook(shutdownHook);
+
+      @Nullable
+      State result;
+      try {
+        result = job.waitToFinish(
+            BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
+            new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+      } catch (IOException | InterruptedException ex) {
+        if (ex instanceof InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
+        throw new DataflowServiceException(
+            job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
+      }
+
+      if (result == null) {
+        throw new DataflowServiceException(
+            job, "Timed out while retrieving status for job " + job.getJobId());
+      }
+
+      LOG.info("Job finished with status {}", result);
+      if (!result.isTerminal()) {
+        throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
+            + ", got " + result);
+      }
+
+      if (result == State.DONE) {
+        return job;
+      } else if (result == State.UPDATED) {
+        DataflowPipelineJob newJob = job.getReplacedByJob();
+        LOG.info("Job {} has been updated and is running as the new job with id {}."
+            + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
+            job.getJobId(),
+            newJob.getJobId(),
+            MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
+        throw new DataflowJobUpdatedException(
+            job,
+            String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
+            newJob);
+      } else if (result == State.CANCELLED) {
+        String message = String.format("Job %s cancelled by user", job.getJobId());
+        LOG.info(message);
+        throw new DataflowJobCancelledException(job, message);
+      } else {
+        throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
+            + " failed with status " + result);
+      }
+    } finally {
+      Runtime.getRuntime().removeShutdownHook(shutdownHook);
+    }
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    return dataflowPipelineRunner.apply(transform, input);
+  }
+
+  /**
+   * Sets callbacks to invoke during execution. See {@link DataflowPipelineRunnerHooks}.
+   */
+  @Experimental
+  public void setHooks(DataflowPipelineRunnerHooks hooks) {
+    this.dataflowPipelineRunner.setHooks(hooks);
+  }
+
+  @Override
+  public String toString() {
+    return "BlockingDataflowPipelineRunner#" + options.getJobName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
new file mode 100644
index 0000000..4a4f100
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
@@ -0,0 +1,35 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+/**
+ * An exception that is thrown if the unique job name constraint of the Dataflow
+ * service is broken because an existing job with the same job name is currently active.
+ * The {@link DataflowPipelineJob} contained within this exception contains information
+ * about the pre-existing job.
+ */
+public class DataflowJobAlreadyExistsException extends DataflowJobException {
+  /**
+   * Create a new {@code DataflowJobAlreadyExistsException} with the specified {@link
+   * DataflowPipelineJob} and message.
+   */
+  public DataflowJobAlreadyExistsException(
+      DataflowPipelineJob job, String message) {
+    super(job, message, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
new file mode 100644
index 0000000..1f52c6a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
@@ -0,0 +1,34 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+/**
+ * An exception that is thrown if the existing job has already been updated within the Dataflow
+ * service and is no longer able to be updated. The {@link DataflowPipelineJob} contained within
+ * this exception contains information about the pre-existing updated job.
+ */
+public class DataflowJobAlreadyUpdatedException extends DataflowJobException {
+  /**
+   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+   * DataflowPipelineJob} and message.
+   */
+  public DataflowJobAlreadyUpdatedException(
+      DataflowPipelineJob job, String message) {
+    super(job, message, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
new file mode 100644
index 0000000..495ca5a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
@@ -0,0 +1,39 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ */
+public class DataflowJobCancelledException extends DataflowJobException {
+  /**
+   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+   * DataflowPipelineJob} and message.
+   */
+  public DataflowJobCancelledException(DataflowPipelineJob job, String message) {
+    super(job, message, null);
+  }
+
+  /**
+   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+   * DataflowPipelineJob}, message, and cause.
+   */
+  public DataflowJobCancelledException(DataflowPipelineJob job, String message, Throwable cause) {
+    super(job, message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
new file mode 100644
index 0000000..a22d13c
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.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 com.google.cloud.dataflow.sdk.runners;
+
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link RuntimeException} that contains information about a {@link DataflowPipelineJob}.
+ */
+public abstract class DataflowJobException extends RuntimeException {
+  private final DataflowPipelineJob job;
+
+  DataflowJobException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+    super(message, cause);
+    this.job = Objects.requireNonNull(job);
+  }
+
+  /**
+   * Returns the failed job.
+   */
+  public DataflowPipelineJob getJob() {
+    return job;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
new file mode 100644
index 0000000..3dbb007
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
@@ -0,0 +1,35 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import javax.annotation.Nullable;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} fails during execution, and
+ * provides access to the failed job.
+ */
+public class DataflowJobExecutionException extends DataflowJobException {
+  DataflowJobExecutionException(DataflowPipelineJob job, String message) {
+    this(job, message, null);
+  }
+
+  DataflowJobExecutionException(
+      DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+    super(job, message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
new file mode 100644
index 0000000..5792941
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
@@ -0,0 +1,51 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ */
+public class DataflowJobUpdatedException extends DataflowJobException {
+  private DataflowPipelineJob replacedByJob;
+
+  /**
+   * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
+   * DataflowPipelineJob}, message, and replacement {@link DataflowPipelineJob}.
+   */
+  public DataflowJobUpdatedException(
+      DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob) {
+    this(job, message, replacedByJob, null);
+  }
+
+  /**
+   * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
+   * DataflowPipelineJob}, message, replacement {@link DataflowPipelineJob}, and cause.
+   */
+  public DataflowJobUpdatedException(
+      DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob, Throwable cause) {
+    super(job, message, cause);
+    this.replacedByJob = replacedByJob;
+  }
+
+  /**
+   * The new job that replaces the job terminated with this exception.
+   */
+  public DataflowPipelineJob getReplacedByJob() {
+    return replacedByJob;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
new file mode 100644
index 0000000..7b97c7d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
@@ -0,0 +1,60 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+
+/**
+ * A {@link DataflowPipeline} is a {@link Pipeline} that returns a
+ * {@link DataflowPipelineJob} when it is
+ * {@link com.google.cloud.dataflow.sdk.Pipeline#run()}.
+ *
+ * <p>This is not intended for use by users of Cloud Dataflow.
+ * Instead, use {@link Pipeline#create(PipelineOptions)} to initialize a
+ * {@link Pipeline}.
+ */
+public class DataflowPipeline extends Pipeline {
+
+  /**
+   * Creates and returns a new {@link DataflowPipeline} instance for tests.
+   */
+  public static DataflowPipeline create(DataflowPipelineOptions options) {
+    return new DataflowPipeline(options);
+  }
+
+  private DataflowPipeline(DataflowPipelineOptions options) {
+    super(DataflowPipelineRunner.fromOptions(options), options);
+  }
+
+  @Override
+  public DataflowPipelineJob run() {
+    return (DataflowPipelineJob) super.run();
+  }
+
+  @Override
+  public DataflowPipelineRunner getRunner() {
+    return (DataflowPipelineRunner) super.getRunner();
+  }
+
+  @Override
+  public String toString() {
+    return "DataflowPipeline#" + getOptions().as(DataflowPipelineOptions.class).getJobName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
new file mode 100644
index 0000000..632be6d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
@@ -0,0 +1,394 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.NanoClock;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowMetricUpdateExtractor;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
+import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
+import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A DataflowPipelineJob represents a job submitted to Dataflow using
+ * {@link DataflowPipelineRunner}.
+ */
+public class DataflowPipelineJob implements PipelineResult {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);
+
+  /**
+   * The id for the job.
+   */
+  private String jobId;
+
+  /**
+   * Google cloud project to associate this pipeline with.
+   */
+  private String projectId;
+
+  /**
+   * Client for the Dataflow service. This can be used to query the service
+   * for information about the job.
+   */
+  private Dataflow dataflowClient;
+
+  /**
+   * The state the job terminated in or {@code null} if the job has not terminated.
+   */
+  @Nullable
+  private State terminalState = null;
+
+  /**
+   * The job that replaced this one or {@code null} if the job has not been replaced.
+   */
+  @Nullable
+  private DataflowPipelineJob replacedByJob = null;
+
+  private DataflowAggregatorTransforms aggregatorTransforms;
+
+  /**
+   * The Metric Updates retrieved after the job was in a terminal state.
+   */
+  private List<MetricUpdate> terminalMetricUpdates;
+
+  /**
+   * The polling interval for job status and messages information.
+   */
+  static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
+  static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
+
+  /**
+   * The amount of polling attempts for job status and messages information.
+   */
+  static final int MESSAGES_POLLING_ATTEMPTS = 10;
+  static final int STATUS_POLLING_ATTEMPTS = 5;
+
+  /**
+   * Constructs the job.
+   *
+   * @param projectId the project id
+   * @param jobId the job id
+   * @param dataflowClient the client for the Dataflow Service
+   */
+  public DataflowPipelineJob(String projectId, String jobId, Dataflow dataflowClient,
+      DataflowAggregatorTransforms aggregatorTransforms) {
+    this.projectId = projectId;
+    this.jobId = jobId;
+    this.dataflowClient = dataflowClient;
+    this.aggregatorTransforms = aggregatorTransforms;
+  }
+
+  /**
+   * Get the id of this job.
+   */
+  public String getJobId() {
+    return jobId;
+  }
+
+  /**
+   * Get the project this job exists in.
+   */
+  public String getProjectId() {
+    return projectId;
+  }
+
+  /**
+   * Returns a new {@link DataflowPipelineJob} for the job that replaced this one, if applicable.
+   *
+   * @throws IllegalStateException if called before the job has terminated or if the job terminated
+   * but was not updated
+   */
+  public DataflowPipelineJob getReplacedByJob() {
+    if (terminalState == null) {
+      throw new IllegalStateException("getReplacedByJob() called before job terminated");
+    }
+    if (replacedByJob == null) {
+      throw new IllegalStateException("getReplacedByJob() called for job that was not replaced");
+    }
+    return replacedByJob;
+  }
+
+  /**
+   * Get the Cloud Dataflow API Client used by this job.
+   */
+  public Dataflow getDataflowClient() {
+    return dataflowClient;
+  }
+
+  /**
+   * Waits for the job to finish and return the final status.
+   *
+   * @param timeToWait The time to wait in units timeUnit for the job to finish.
+   *     Provide a value less than 1 ms for an infinite wait.
+   * @param timeUnit The unit of time for timeToWait.
+   * @param messageHandler If non null this handler will be invoked for each
+   *   batch of messages received.
+   * @return The final state of the job or null on timeout or if the
+   *   thread is interrupted.
+   * @throws IOException If there is a persistent problem getting job
+   *   information.
+   * @throws InterruptedException
+   */
+  @Nullable
+  public State waitToFinish(
+      long timeToWait,
+      TimeUnit timeUnit,
+      MonitoringUtil.JobMessagesHandler messageHandler)
+          throws IOException, InterruptedException {
+    return waitToFinish(timeToWait, timeUnit, messageHandler, Sleeper.DEFAULT, NanoClock.SYSTEM);
+  }
+
+  /**
+   * Wait for the job to finish and return the final status.
+   *
+   * @param timeToWait The time to wait in units timeUnit for the job to finish.
+   *     Provide a value less than 1 ms for an infinite wait.
+   * @param timeUnit The unit of time for timeToWait.
+   * @param messageHandler If non null this handler will be invoked for each
+   *   batch of messages received.
+   * @param sleeper A sleeper to use to sleep between attempts.
+   * @param nanoClock A nanoClock used to time the total time taken.
+   * @return The final state of the job or null on timeout or if the
+   *   thread is interrupted.
+   * @throws IOException If there is a persistent problem getting job
+   *   information.
+   * @throws InterruptedException
+   */
+  @Nullable
+  @VisibleForTesting
+  State waitToFinish(
+      long timeToWait,
+      TimeUnit timeUnit,
+      MonitoringUtil.JobMessagesHandler messageHandler,
+      Sleeper sleeper,
+      NanoClock nanoClock)
+          throws IOException, InterruptedException {
+    MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowClient);
+
+    long lastTimestamp = 0;
+    BackOff backoff =
+        timeUnit.toMillis(timeToWait) > 0
+            ? new AttemptAndTimeBoundedExponentialBackOff(
+                MESSAGES_POLLING_ATTEMPTS,
+                MESSAGES_POLLING_INTERVAL,
+                timeUnit.toMillis(timeToWait),
+                AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
+                nanoClock)
+            : new AttemptBoundedExponentialBackOff(
+                MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL);
+    State state;
+    do {
+      // Get the state of the job before listing messages. This ensures we always fetch job
+      // messages after the job finishes to ensure we have all them.
+      state = getStateWithRetries(1, sleeper);
+      boolean hasError = state == State.UNKNOWN;
+
+      if (messageHandler != null && !hasError) {
+        // Process all the job messages that have accumulated so far.
+        try {
+          List<JobMessage> allMessages = monitor.getJobMessages(
+              jobId, lastTimestamp);
+
+          if (!allMessages.isEmpty()) {
+            lastTimestamp =
+                fromCloudTime(allMessages.get(allMessages.size() - 1).getTime()).getMillis();
+            messageHandler.process(allMessages);
+          }
+        } catch (GoogleJsonResponseException | SocketTimeoutException e) {
+          hasError = true;
+          LOG.warn("There were problems getting current job messages: {}.", e.getMessage());
+          LOG.debug("Exception information:", e);
+        }
+      }
+
+      if (!hasError) {
+        backoff.reset();
+        // Check if the job is done.
+        if (state.isTerminal()) {
+          return state;
+        }
+      }
+    } while(BackOffUtils.next(sleeper, backoff));
+    LOG.warn("No terminal state was returned.  State value {}", state);
+    return null;  // Timed out.
+  }
+
+  /**
+   * Cancels the job.
+   * @throws IOException if there is a problem executing the cancel request.
+   */
+  public void cancel() throws IOException {
+    Job content = new Job();
+    content.setProjectId(projectId);
+    content.setId(jobId);
+    content.setRequestedState("JOB_STATE_CANCELLED");
+    dataflowClient.projects().jobs()
+        .update(projectId, jobId, content)
+        .execute();
+  }
+
+  @Override
+  public State getState() {
+    if (terminalState != null) {
+      return terminalState;
+    }
+
+    return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT);
+  }
+
+  /**
+   * Attempts to get the state. Uses exponential backoff on failure up to the maximum number
+   * of passed in attempts.
+   *
+   * @param attempts The amount of attempts to make.
+   * @param sleeper Object used to do the sleeps between attempts.
+   * @return The state of the job or State.UNKNOWN in case of failure.
+   */
+  @VisibleForTesting
+  State getStateWithRetries(int attempts, Sleeper sleeper) {
+    if (terminalState != null) {
+      return terminalState;
+    }
+    try {
+      Job job = getJobWithRetries(attempts, sleeper);
+      return MonitoringUtil.toState(job.getCurrentState());
+    } catch (IOException exn) {
+      // The only IOException that getJobWithRetries is permitted to throw is the final IOException
+      // that caused the failure of retry. Other exceptions are wrapped in an unchecked exceptions
+      // and will propagate.
+      return State.UNKNOWN;
+    }
+  }
+
+  /**
+   * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the
+   * maximum number of passed in attempts.
+   *
+   * @param attempts The amount of attempts to make.
+   * @param sleeper Object used to do the sleeps between attempts.
+   * @return The underlying {@link Job} object.
+   * @throws IOException When the maximum number of retries is exhausted, the last exception is
+   * thrown.
+   */
+  @VisibleForTesting
+  Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException {
+    AttemptBoundedExponentialBackOff backoff =
+        new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL);
+
+    // Retry loop ends in return or throw
+    while (true) {
+      try {
+        Job job = dataflowClient
+            .projects()
+            .jobs()
+            .get(projectId, jobId)
+            .execute();
+        State currentState = MonitoringUtil.toState(job.getCurrentState());
+        if (currentState.isTerminal()) {
+          terminalState = currentState;
+          replacedByJob = new DataflowPipelineJob(
+              getProjectId(), job.getReplacedByJobId(), dataflowClient, aggregatorTransforms);
+        }
+        return job;
+      } catch (IOException exn) {
+        LOG.warn("There were problems getting current job status: {}.", exn.getMessage());
+        LOG.debug("Exception information:", exn);
+
+        if (!nextBackOff(sleeper, backoff)) {
+          throw exn;
+        }
+      }
+    }
+  }
+
+  /**
+   * Identical to {@link BackOffUtils#next} but without checked exceptions.
+   */
+  private boolean nextBackOff(Sleeper sleeper, BackOff backoff) {
+    try {
+      return BackOffUtils.next(sleeper, backoff);
+    } catch (InterruptedException | IOException e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+      throw Throwables.propagate(e);
+    }
+  }
+
+  @Override
+  public <OutputT> AggregatorValues<OutputT> getAggregatorValues(Aggregator<?, OutputT> aggregator)
+      throws AggregatorRetrievalException {
+    try {
+      return new MapAggregatorValues<>(fromMetricUpdates(aggregator));
+    } catch (IOException e) {
+      throw new AggregatorRetrievalException(
+          "IOException when retrieving Aggregator values for Aggregator " + aggregator, e);
+    }
+  }
+
+  private <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator)
+      throws IOException {
+    if (aggregatorTransforms.contains(aggregator)) {
+      List<MetricUpdate> metricUpdates;
+      if (terminalMetricUpdates != null) {
+        metricUpdates = terminalMetricUpdates;
+      } else {
+        boolean terminal = getState().isTerminal();
+        JobMetrics jobMetrics =
+            dataflowClient.projects().jobs().getMetrics(projectId, jobId).execute();
+        metricUpdates = jobMetrics.getMetrics();
+        if (terminal && jobMetrics.getMetrics() != null) {
+          terminalMetricUpdates = metricUpdates;
+        }
+      }
+
+      return DataflowMetricUpdateExtractor.fromMetricUpdates(
+          aggregator, aggregatorTransforms, metricUpdates);
+    } else {
+      throw new IllegalArgumentException(
+          "Aggregator " + aggregator + " is not used in this pipeline");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
new file mode 100644
index 0000000..8aaa7cc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
@@ -0,0 +1,59 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import com.google.auto.service.AutoService;
+import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for
+ * the {@link DataflowPipeline}.
+ */
+public class DataflowPipelineRegistrar {
+  private DataflowPipelineRegistrar() { }
+
+  /**
+   * Register the {@link DataflowPipelineOptions} and {@link BlockingDataflowPipelineOptions}.
+   */
+  @AutoService(PipelineOptionsRegistrar.class)
+  public static class Options implements PipelineOptionsRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+      return ImmutableList.<Class<? extends PipelineOptions>>of(
+          DataflowPipelineOptions.class,
+          BlockingDataflowPipelineOptions.class);
+    }
+  }
+
+  /**
+   * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}.
+   */
+  @AutoService(PipelineRunnerRegistrar.class)
+  public static class Runner implements PipelineRunnerRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
+          DataflowPipelineRunner.class,
+          BlockingDataflowPipelineRunner.class);
+    }
+  }
+}


[59/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
index 9a84ffe..fe02666 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
 
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
 import com.google.common.base.MoreObjects;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 8fee332..13c1e34 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
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.SetCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.ValueState;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.SetCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MimeTypes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MimeTypes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MimeTypes.java
index 45b3523..b885531 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MimeTypes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MimeTypes.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /** Constants representing various mime types. */
 public class MimeTypes {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetector.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetector.java
index a5b467b..a626153 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetector.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetector.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * An object for detecting illegal mutations.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
index 7f356d2..ee84c45 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
 
 import java.util.Arrays;
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
index e1bb7f1..e809c24 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.util.state.AccumulatorCombiningState;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateMerging;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
 
 /**
  * Tracks which windows have non-empty panes. Specifically, which windows have new elements since

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
index 52d8275..0d02302 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
index e59ab33..09f1f90 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 
 import java.io.IOException;
 import java.security.GeneralSecurityException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
index 4e3864e..feee6a0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java
index 9950868..99319ed 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
 import com.google.common.collect.Sets;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/OutputReference.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/OutputReference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/OutputReference.java
index 015038c..5e30172 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/OutputReference.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/OutputReference.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.api.client.util.Preconditions.checkNotNull;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java
index 811dd06..a7311e3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import java.util.Objects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
index c5e104d..0b9107c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValueBase;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+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.PValueBase;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.HashMultimap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java
index 7c41455..eea7bcd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.Collections;
 import java.util.LinkedHashMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
index bb996f8..5e08031 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.ValueState;
+package org.apache.beam.sdk.util;
+
+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;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
index 329b0b0..a525e94 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * Interface for controlling validation of paths.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java
index 022512b..9dc4f68 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 import java.io.Serializable;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java
index ac13ed7..2d28682 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.RequiresContextInternal;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
 import com.google.common.collect.Iterables;
 
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
index f2bc022..cc9fa5e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * Constant property names used by the SDK in CloudWorkflow specifications.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RandomAccessData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RandomAccessData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RandomAccessData.java
index 9c4f31d..2d902f4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RandomAccessData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RandomAccessData.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.io.ByteStreams;
 import com.google.common.primitives.UnsignedBytes;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
index b7f2f97..c5ee1e1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
index 81f607a..c90940e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
@@ -15,26 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.State;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateContext;
-import com.google.cloud.dataflow.sdk.util.state.StateContexts;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces.WindowNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
index 05c5d7d..fe53389 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
@@ -15,27 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks;
-import com.google.cloud.dataflow.sdk.util.ReduceFnContextFactory.StateStyle;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces.WindowNamespace;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks;
+import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
index 8c484db..c2273f5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.KV;
 
 /**
  * DoFn that makes timestamps and window assignments explicit in the value part of each key/value

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java
index 46c92b2..09b2222 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+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.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
+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.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
index d772566..e5168b2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
index f324e0d..3857435 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.http.HttpBackOffIOExceptionHandler;
 import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c660f70..06b5bb2 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
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.decodeFromByteArray;
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
+import static org.apache.beam.sdk.util.CoderUtils.decodeFromByteArray;
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
 import com.google.common.base.Preconditions;
 
 import org.xerial.snappy.SnappyInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java
index 6f58918..738577d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.DeserializationFeature;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
index 6006e30..52a12bd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java
index 0d19647..e99eb57 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
index 85cbade..78377c8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
index ffd64fa..562ec3f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f8d75df..fbe54fa 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.PTransform;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java
index 20d56a0..d6b704b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.util.Data;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
index 765b428..9a42b23 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
index 6183124..2eeee54 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
@@ -15,24 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.state.AccumulatorCombiningState;
-import com.google.cloud.dataflow.sdk.util.state.BagState;
-import com.google.cloud.dataflow.sdk.util.state.CombiningState;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateMerging;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
 
 /**
  * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java
index 28b524d..554ac1b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.auth.oauth2.BearerToken;
 import com.google.api.client.auth.oauth2.Credential;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java
index c9b8110..f03446e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * {@code TimeDomain} specifies whether an operation is based on

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeUtil.java
index f43d1b7..db5c760 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeUtil.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeUtil.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import org.joda.time.DateTime;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0f14829..5f67131 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
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java
index 2cd04a6..e8404bd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import org.apache.beam.sdk.annotations.Experimental;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
index d5cdb24..9082ce3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java
@@ -15,7 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PubsubOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
@@ -26,9 +30,6 @@ import com.google.api.client.json.jackson2.JacksonFactory;
 import com.google.api.services.bigquery.Bigquery;
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.storage.Storage;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PubsubOptions;
 import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ce19162..4855654 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
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.MergingTriggerInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerInfo;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.State;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+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.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+
 import com.google.common.base.Predicate;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
index 24e92f8..f104f6a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.ValueState;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java
index ecfa983..58c6617 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.common.base.MoreObjects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java
index ecf1d68..c2f30c6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.common.base.MoreObjects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java
index c515e26..caa0759 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.http.GenericUrl;
 import com.google.api.client.http.HttpResponse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
index 13116dc..14443a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.util.Arrays;
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 afeb3b9..8e4e134 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
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.util;
+
+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 org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java
index 2ba3e88..71110e1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java
index 8ef8444..57a117a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.EOFException;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
index 916704e..eb9c257 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.ReadableState;
-import com.google.cloud.dataflow.sdk.util.state.StateMerging;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.WatermarkHoldState;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Weighted.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Weighted.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Weighted.java
index 418fcd6..b23404f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Weighted.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Weighted.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * Interface representing an object that has a weight, in unspecified units.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
index 3a3f7d1..6197454 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * A {@code T} with an accompanying weight. Units are unspecified.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
index 886746c..d6940a2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 4a4cb3e..f6e82cf 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
@@ -15,22 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.util.Structs.addBoolean;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CollectionCoder;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CollectionCoder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+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.PaneInfo.PaneInfoCoder;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
index 65e4f52..149c497 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java
index a94cce3..a82f2b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+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.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+
 import com.google.common.base.MoreObjects;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 43b454d..6d73027 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java
index f02609e..6cdacc5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java
@@ -15,14 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
+
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR;
 
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.AND;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.OR;
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.util.concurrent.AtomicDouble;
 
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java
index a7cfe12..c2550cd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 /**
  * A counter provider can provide {@link Counter} instances.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java
index a5b7255..699d7d3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import static com.google.common.base.Preconditions.checkArgument;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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
index eb1a14e..613aa4b 100644
--- 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 /**
  * An interface for things that allow observing the size in bytes of

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterable.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterable.java
index f6eca45..ee3c43e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterable.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterable.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterator.java
index efa787d..7f2e40c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservableIterator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import java.util.Iterator;
 import java.util.Observable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
index 56d671e..55d8b07 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import java.util.Observable;
 import java.util.Observer;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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
index 1a092e6..1e3c17f 100644
--- 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 33dec02..3201271 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import static java.util.Arrays.asList;
 



[66/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
index bf618c4..ea125de 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
@@ -26,23 +26,21 @@ import java.util.Map;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
-
 import org.apache.beam.runners.spark.aggregators.AggAccumParam;
 import org.apache.beam.runners.spark.aggregators.NamedAggregators;
-
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.TypeDescriptor;
 import org.apache.spark.Accumulator;
 import org.apache.spark.api.java.JavaSparkContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
index d8481bf..30ab076 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java
@@ -20,7 +20,7 @@ package org.apache.beam.runners.spark.translation;
 
 import java.io.Serializable;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 public interface TransformEvaluator<PT extends PTransform<?, ?>> extends Serializable {
   void evaluate(PT transform, EvaluationContext context);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
index 8fe3b24..0366856 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
@@ -18,6 +18,11 @@
 
 package org.apache.beam.runners.spark.translation;
 
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
+import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.Arrays;
@@ -25,40 +30,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
-import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-
 import org.apache.avro.mapred.AvroKey;
 import org.apache.avro.mapreduce.AvroJob;
 import org.apache.avro.mapreduce.AvroKeyInputFormat;
@@ -69,6 +43,31 @@ import org.apache.beam.runners.spark.io.hadoop.TemplatedAvroKeyOutputFormat;
 import org.apache.beam.runners.spark.io.hadoop.TemplatedTextOutputFormat;
 import org.apache.beam.runners.spark.util.BroadcastHelper;
 import org.apache.beam.runners.spark.util.ByteArray;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.AssignWindowsDoFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -82,7 +81,6 @@ import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.Function2;
 import org.apache.spark.api.java.function.PairFlatMapFunction;
 import org.apache.spark.api.java.function.PairFunction;
-
 import scala.Tuple2;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
index 6ba04b7..e92b6d1 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java
@@ -18,7 +18,7 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.spark.api.java.function.Function;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
index 2c34caf..4abf816 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsFactory.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 public final class SparkStreamingPipelineOptionsFactory {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
index e39d3ed..2e35098 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkStreamingPipelineOptionsRegistrar.java
@@ -17,10 +17,10 @@
  */
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 
 public class SparkStreamingPipelineOptionsRegistrar implements PipelineOptionsRegistrar {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
index b35c47c..1c83700 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
@@ -25,18 +25,16 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
 import org.apache.beam.runners.spark.translation.EvaluationContext;
 import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
-
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaRDDLike;
 import org.apache.spark.api.java.JavaSparkContext;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
index 711358e..d9daeb0 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -27,29 +27,8 @@ import java.util.Set;
 import com.google.api.client.util.Lists;
 import com.google.api.client.util.Maps;
 import com.google.api.client.util.Sets;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
 import com.google.common.reflect.TypeToken;
 import kafka.serializer.Decoder;
-
 import org.apache.beam.runners.spark.io.ConsoleIO;
 import org.apache.beam.runners.spark.io.CreateStream;
 import org.apache.beam.runners.spark.io.KafkaIO;
@@ -60,7 +39,25 @@ import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
 import org.apache.beam.runners.spark.translation.TransformEvaluator;
 import org.apache.beam.runners.spark.translation.TransformTranslator;
 import org.apache.beam.runners.spark.translation.WindowingHelpers;
-
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.AssignWindowsDoFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PDone;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.VoidFunction;
@@ -71,7 +68,6 @@ import org.apache.spark.streaming.api.java.JavaDStreamLike;
 import org.apache.spark.streaming.api.java.JavaPairInputDStream;
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
 import org.apache.spark.streaming.kafka.KafkaUtils;
-
 import scala.Tuple2;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
index 504ea92..8c018d3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingWindowPipelineDetector.java
@@ -18,20 +18,18 @@
 
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
 import org.apache.beam.runners.spark.SparkPipelineRunner;
 import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
 import org.apache.beam.runners.spark.translation.TransformTranslator;
-
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 import org.apache.spark.streaming.Duration;
 import org.apache.spark.streaming.Durations;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
index c3588c9..1824a9d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java
@@ -22,8 +22,8 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.Serializable;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
 import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.broadcast.Broadcast;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
index 87f6d4c..d13404c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java
@@ -18,20 +18,20 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
-import org.junit.Test;
-
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
 /**
  * A test based on {@code DeDupExample} from the SDK.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
index 057cf3b..eeca10f 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java
@@ -18,20 +18,20 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Iterables;
-import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 import java.util.Collections;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
 
 public class EmptyInputTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 3946e12..c413b3f 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
@@ -18,21 +18,27 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
-import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
-import org.junit.Test;
-
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.ImmutableSet;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+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.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
 public class SimpleWordCountTest {
   private static final String[] WORDS_ARRAY = {
       "hi there", "hi", "hi sue bob",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
index 24183fb..4e80fe9 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java
@@ -18,21 +18,21 @@
 
 package org.apache.beam.runners.spark;
 
-import com.google.cloud.dataflow.examples.complete.TfIdf;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import org.junit.Test;
-
 import java.net.URI;
 import java.util.Arrays;
 
+import org.apache.beam.examples.complete.TfIdf;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
 /**
  * A test based on {@code TfIdf} from the SDK.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
index 8ba64ec..538fd97 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.beam.runners.spark.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
index b8e2697..f8f9624 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
@@ -18,16 +18,15 @@
 
 package org.apache.beam.runners.spark.io;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.Lists;
-import com.google.common.io.Resources;
+import static org.junit.Assert.assertEquals;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.List;
+
+import com.google.common.collect.Lists;
+import com.google.common.io.Resources;
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.file.DataFileWriter;
@@ -37,13 +36,15 @@ import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.PCollection;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import static org.junit.Assert.assertEquals;
-
 public class AvroPipelineTest {
 
   private File inputFile;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
index 9edf41c..37a7d54 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
@@ -18,35 +18,35 @@
 
 package org.apache.beam.runners.spark.io;
 
-import com.google.cloud.dataflow.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
 import com.google.common.base.Charsets;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
+import org.apache.beam.examples.WordCount;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
+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.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 public class NumShardsTest {
 
   private static final String[] WORDS_ARRAY = {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
index 9e5fdbc..b7df739 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java
@@ -18,14 +18,19 @@
 
 package org.apache.beam.runners.spark.io.hadoop;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
 import org.apache.beam.runners.spark.coders.WritableCoder;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -41,11 +46,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
 public class HadoopFileFormatPipelineTest {
 
   private File inputFile;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
index 90053ac..eecfd58 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.beam.runners.spark.io.hadoop;
 
-import org.junit.Test;
-
 import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputDirectory;
 import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix;
 import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
@@ -27,6 +25,8 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceSh
 import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber;
 import static org.junit.Assert.assertEquals;
 
+import org.junit.Test;
+
 public class ShardNameBuilderTest {
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
index 6945d68..bd90aae 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java
@@ -18,22 +18,22 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
 import com.google.common.collect.Iterables;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
 import org.junit.Test;
 
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
 public class CombineGloballyTest {
 
   private static final String[] WORDS_ARRAY = {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
index 0373968..6cf9330 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java
@@ -18,24 +18,28 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+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.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 public class CombinePerKeyTest {
 
     private static final List<String> WORDS =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
index 5f15a35..80b5fc5 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java
@@ -18,19 +18,19 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import java.io.Serializable;
+
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
 import org.junit.Test;
 
-import java.io.Serializable;
-
 public class DoFnOutputTest implements Serializable {
   @Test
   public void test() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
index 52dcf30..1450a8f 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java
@@ -18,22 +18,38 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.*;
+import java.util.Set;
+
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.ApproximateUnique;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.Set;
-
 public class MultiOutputWordCountTest {
 
   private static final TupleTag<String> upper = new TupleTag<>();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
index 0978766..2237ea2 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
@@ -18,22 +18,6 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import org.apache.beam.runners.spark.EvaluationResult;
-import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.junit.Test;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -42,6 +26,28 @@ import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.apache.beam.runners.spark.EvaluationResult;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+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.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
 public class SerializationTest {
 
   public static class StringHolder { // not serializable

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
index fc14fc7..b85d935 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java
@@ -18,22 +18,24 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.Serializable;
+import java.net.URI;
+
 import org.apache.beam.runners.spark.SparkPipelineOptions;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.Serializable;
-import java.net.URI;
-
-import static org.junit.Assert.*;
-
 public class SideEffectsTest implements Serializable {
 
   static class UserException extends RuntimeException {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
index da30321..f07fa0b 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
@@ -18,15 +18,22 @@
 
 package org.apache.beam.runners.spark.translation;
 
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.List;
+
 import com.google.api.client.repackaged.com.google.common.base.Joiner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Charsets;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.commons.io.FileUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -34,13 +41,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.Collections;
-import java.util.List;
-
 /**
  * A test for the transforms registered in TransformTranslator.
  * Builds a regular Dataflow pipeline with each of the mapped

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
index 9a29171..5134b5c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java
@@ -18,23 +18,22 @@
 
 package org.apache.beam.runners.spark.translation;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SimpleWordCountTest;
-import org.apache.beam.runners.spark.SparkPipelineOptions;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
index ed51edf..17f7fbd 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java
@@ -17,29 +17,27 @@
  */
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 
-import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
-import org.apache.beam.runners.spark.io.CreateStream;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.CreateStream;
 import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
-
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.View;
+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.joda.time.Duration;
 import org.junit.Test;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
 /**
  * Test Flatten (union) implementation for streaming.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
index 4499b56..a0c8a4e 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
@@ -17,27 +17,32 @@
  */
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
-import org.apache.beam.runners.spark.io.KafkaIO;
+import kafka.serializer.StringDecoder;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
-import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.KafkaIO;
 import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster;
-
+import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+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.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.Serializer;
@@ -47,14 +52,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import kafka.serializer.StringDecoder;
-
 /**
  * Test Kafka as input.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
index ed02286..de554e2 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
@@ -17,30 +17,28 @@
  */
 package org.apache.beam.runners.spark.translation.streaming;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableSet;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
 
-import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
-import org.apache.beam.runners.spark.io.CreateStream;
+import com.google.common.collect.ImmutableSet;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SimpleWordCountTest;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
+import org.apache.beam.runners.spark.SparkStreamingPipelineOptions;
+import org.apache.beam.runners.spark.io.CreateStream;
 import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
-
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.View;
+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.joda.time.Duration;
 import org.junit.Test;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
 public class SimpleStreamingWordCountTest {
 
   private static final String[] WORDS_ARRAY = {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
index e967cdb..21c8115 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
@@ -17,10 +17,6 @@
  */
 package org.apache.beam.runners.spark.translation.streaming.utils;
 
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -35,6 +31,9 @@ import java.util.Random;
 import kafka.server.KafkaConfig;
 import kafka.server.KafkaServer;
 import kafka.utils.Time;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
index fe0f2a8..3f50798 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
@@ -18,7 +18,6 @@
 package org.apache.beam.runners.spark.translation.streaming.utils;
 
 import org.apache.beam.runners.spark.EvaluationResult;
-
 import org.junit.Assert;
 
 /**
@@ -27,7 +26,7 @@ import org.junit.Assert;
  */
 public final class PAssertStreaming {
   /**
-   * Copied aggregator names from {@link com.google.cloud.dataflow.sdk.testing.PAssert}
+   * Copied aggregator names from {@link org.apache.beam.sdk.testing.PAssert}
    */
   static final String SUCCESS_COUNTER = "PAssertSuccess";
   static final String FAILURE_COUNTER = "PAssertFailure";

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index a89973e..859e07a 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -148,8 +148,8 @@
           <doctitle>Google Cloud Dataflow SDK for Java, version ${project.version}</doctitle>
           <overview>../javadoc/overview.html</overview>
 
-          <subpackages>com.google.cloud.dataflow.sdk</subpackages>
-          <additionalparam>-exclude com.google.cloud.dataflow.sdk.runners.worker:com.google.cloud.dataflow.sdk.runners.dataflow:com.google.cloud.dataflow.sdk.util:com.google.cloud.dataflow.sdk.runners.inprocess ${dataflow.javadoc_opts}</additionalparam>
+          <subpackages>org.apache.beam.sdk</subpackages>
+          <additionalparam>-exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess ${dataflow.javadoc_opts}</additionalparam>
           <use>false</use>
           <quiet>true</quiet>
           <bottom><![CDATA[<br>]]></bottom>
@@ -243,19 +243,19 @@
               </filters>
               <relocations>
                 <!-- TODO: Once ready, change the following pattern to 'com'
-                     only, exclude 'com.google.cloud.dataflow.**', and remove
+                     only, exclude 'org.apache.beam.**', and remove
                      the second relocation. -->
                 <relocation>
                   <pattern>com.google.common</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.common</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.common</shadedPattern>
                 </relocation>
                 <relocation>
                   <pattern>com.google.thirdparty</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.thirdparty</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.thirdparty</shadedPattern>
                 </relocation>
                 <relocation>
                   <pattern>com.google.cloud.bigtable</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.cloud.bigtable</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.cloud.bigtable</shadedPattern>
                   <excludes>
                     <exclude>com.google.cloud.bigtable.config.BigtableOptions*</exclude>
                     <exclude>com.google.cloud.bigtable.config.CredentialOptions*</exclude>
@@ -674,7 +674,7 @@
     </dependency>
 
     <!--
-    To use com.google.cloud.dataflow.io.XmlSource:
+    To use org.apache.beam.io.XmlSource:
 
     1. Explicitly declare the following dependency for the stax2 API.
     2. Include a stax2 implementation on the classpath. One example
@@ -703,7 +703,7 @@
     </dependency>
 
     <!--
-    To use com.google.cloud.dataflow.io.AvroSource with XZ-encoded files,
+    To use org.apache.beam.io.AvroSource with XZ-encoded files,
     please explicitly declare this dependency to include org.tukaani:xz on
     the classpath at runtime.
     -->

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 db20d41..65a0755 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
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
-
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformHierarchy;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
+package org.apache.beam.sdk;
+
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.UserCodeException;
+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 org.apache.beam.sdk.values.PValue;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
index 90d09ec..f67cb47 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;
 
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
 
 /**
  * Result of {@link Pipeline#run()}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
index adda5a2..60908fa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import java.util.Collections;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
index e7ac0e3..3b93ec3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addString;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -100,7 +100,7 @@ import javax.annotation.Nullable;
  * <p>The implementation attempts to determine if the Avro encoding of the given type will satisfy
  * the criteria of {@link Coder#verifyDeterministic} by inspecting both the type and the
  * Schema provided or generated by Avro. Only coders that are deterministic can be used in
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operations.
+ * {@link org.apache.beam.sdk.transforms.GroupByKey} operations.
  *
  * @param <T> the type of elements handled by this coder
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java
index aea6dd5..bd3bcf3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java
index a0325ac..5f5793e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java
index 6b00a0e..4115825 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.ExposedByteArrayOutputStream;
+import org.apache.beam.sdk.util.StreamUtils;
+import org.apache.beam.sdk.util.VarInt;
 
-import com.google.cloud.dataflow.sdk.util.ExposedByteArrayOutputStream;
-import com.google.cloud.dataflow.sdk.util.StreamUtils;
-import com.google.cloud.dataflow.sdk.util.VarInt;
 import com.google.common.io.ByteStreams;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java
index df9c0c8..e146945 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java
index 557d2bd..b03f98a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.VarInt;
 
-import com.google.cloud.dataflow.sdk.util.VarInt;
 import com.google.common.io.ByteStreams;
 import com.google.protobuf.ByteString;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CannotProvideCoderException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CannotProvideCoderException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CannotProvideCoderException.java
index 66efefa..b490fda 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CannotProvideCoderException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CannotProvideCoderException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 /**
  * The exception thrown when a {@link CoderProvider} cannot

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ff2e10e..71e99f7 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.util.CloudObject;
+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;
@@ -207,7 +208,7 @@ public interface Coder<T> extends Serializable {
    * constant time (or lazily).
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
+   * {@link org.apache.beam.sdk.runners.PipelineRunner}
    * implementations.
    */
   public boolean isRegisterByteSizeObserverCheap(T value, Context context);
@@ -217,7 +218,7 @@ public interface Coder<T> extends Serializable {
    * of the encoded value using this {@code Coder}.
    *
    * <p>Not intended to be called by user code, but instead by
-   * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
+   * {@link org.apache.beam.sdk.runners.PipelineRunner}
    * implementations.
    */
   public void registerByteSizeObserver(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderException.java
index f7bbb69..02232ce 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java
index 7f788c7..5bb56b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactory.java
index 775bfc6..b49cdbd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProvider.java
index f8e29e6..0289447 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProvider.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
  * A {@link CoderProvider} may create a {@link Coder} for

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
index 64d258d..3b852e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;



[51/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
Rename com/google/cloud/dataflow->org/apache/beam

Applied this script:

    mkdir -p $1/org/apache/beam;
    git mv $1/com/google/cloud/dataflow/* $1/org/apache;
    rmdir -p $1/com/google/cloud/dataflow;

To these directories:
  - sdks/java/core/src/{main,test}
  - sdks/java/java8tests/src/test
  - runners/google-cloud-dataflow-java/src/{main,test}
  - contrib/{join-library,hadoop}/src/{main,test}
  - examples/java{,8}/src/{main,test}


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

Branch: refs/heads/master
Commit: 0393a7917318baaa1e580259a74bff2c1dcbe6b8
Parents: c4cbbb1
Author: bchambers <bc...@google.com>
Authored: Wed Apr 13 14:18:40 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Wed Apr 13 17:34:42 2016 -0700

----------------------------------------------------------------------
 .../contrib/hadoop/HadoopFileSource.java        |  486 ---
 .../dataflow/contrib/hadoop/WritableCoder.java  |  111 -
 .../apache/contrib/hadoop/HadoopFileSource.java |  486 +++
 .../apache/contrib/hadoop/WritableCoder.java    |  111 +
 .../contrib/hadoop/HadoopFileSourceTest.java    |  190 --
 .../contrib/hadoop/WritableCoderTest.java       |   37 -
 .../contrib/hadoop/HadoopFileSourceTest.java    |  190 ++
 .../contrib/hadoop/WritableCoderTest.java       |   37 +
 .../dataflow/contrib/joinlibrary/Join.java      |  186 --
 .../org/apache/contrib/joinlibrary/Join.java    |  186 ++
 .../contrib/joinlibrary/InnerJoinTest.java      |  143 -
 .../contrib/joinlibrary/OuterLeftJoinTest.java  |  153 -
 .../contrib/joinlibrary/OuterRightJoinTest.java |  153 -
 .../contrib/joinlibrary/InnerJoinTest.java      |  143 +
 .../contrib/joinlibrary/OuterLeftJoinTest.java  |  153 +
 .../contrib/joinlibrary/OuterRightJoinTest.java |  153 +
 .../dataflow/examples/DebuggingWordCount.java   |  199 --
 .../dataflow/examples/MinimalWordCount.java     |  118 -
 .../dataflow/examples/WindowedWordCount.java    |  270 --
 .../cloud/dataflow/examples/WordCount.java      |  207 --
 .../examples/common/DataflowExampleOptions.java |   37 -
 .../examples/common/DataflowExampleUtils.java   |  488 ---
 .../common/ExampleBigQueryTableOptions.java     |   56 -
 ...xamplePubsubTopicAndSubscriptionOptions.java |   47 -
 .../common/ExamplePubsubTopicOptions.java       |   47 -
 .../examples/common/PubsubFileInjector.java     |  154 -
 .../examples/complete/AutoComplete.java         |  517 ---
 .../cloud/dataflow/examples/complete/README.md  |   44 -
 .../examples/complete/StreamingWordExtract.java |  164 -
 .../cloud/dataflow/examples/complete/TfIdf.java |  432 ---
 .../examples/complete/TopWikipediaSessions.java |  224 --
 .../examples/complete/TrafficMaxLaneFlow.java   |  426 ---
 .../examples/complete/TrafficRoutes.java        |  460 ---
 .../examples/cookbook/BigQueryTornadoes.java    |  180 --
 .../cookbook/CombinePerKeyExamples.java         |  224 --
 .../examples/cookbook/DatastoreWordCount.java   |  270 --
 .../examples/cookbook/DeDupExample.java         |  101 -
 .../examples/cookbook/FilterExamples.java       |  267 --
 .../examples/cookbook/JoinExamples.java         |  186 --
 .../examples/cookbook/MaxPerKeyExamples.java    |  174 -
 .../cloud/dataflow/examples/cookbook/README.md  |   55 -
 .../examples/cookbook/TriggerExample.java       |  565 ----
 .../beam/examples/DebuggingWordCount.java       |  199 ++
 .../apache/beam/examples/MinimalWordCount.java  |  118 +
 .../apache/beam/examples/WindowedWordCount.java |  270 ++
 .../org/apache/beam/examples/WordCount.java     |  207 ++
 .../examples/common/DataflowExampleOptions.java |   37 +
 .../examples/common/DataflowExampleUtils.java   |  488 +++
 .../common/ExampleBigQueryTableOptions.java     |   56 +
 ...xamplePubsubTopicAndSubscriptionOptions.java |   47 +
 .../common/ExamplePubsubTopicOptions.java       |   47 +
 .../examples/common/PubsubFileInjector.java     |  154 +
 .../beam/examples/complete/AutoComplete.java    |  517 +++
 .../org/apache/beam/examples/complete/README.md |   44 +
 .../examples/complete/StreamingWordExtract.java |  164 +
 .../apache/beam/examples/complete/TfIdf.java    |  432 +++
 .../examples/complete/TopWikipediaSessions.java |  224 ++
 .../examples/complete/TrafficMaxLaneFlow.java   |  426 +++
 .../beam/examples/complete/TrafficRoutes.java   |  460 +++
 .../examples/cookbook/BigQueryTornadoes.java    |  180 ++
 .../cookbook/CombinePerKeyExamples.java         |  224 ++
 .../examples/cookbook/DatastoreWordCount.java   |  270 ++
 .../beam/examples/cookbook/DeDupExample.java    |  101 +
 .../beam/examples/cookbook/FilterExamples.java  |  267 ++
 .../beam/examples/cookbook/JoinExamples.java    |  186 ++
 .../examples/cookbook/MaxPerKeyExamples.java    |  174 +
 .../org/apache/beam/examples/cookbook/README.md |   55 +
 .../beam/examples/cookbook/TriggerExample.java  |  565 ++++
 .../examples/DebuggingWordCountTest.java        |   46 -
 .../cloud/dataflow/examples/WordCountTest.java  |   86 -
 .../examples/complete/AutoCompleteTest.java     |  182 --
 .../dataflow/examples/complete/TfIdfTest.java   |   68 -
 .../complete/TopWikipediaSessionsTest.java      |   63 -
 .../cookbook/BigQueryTornadoesTest.java         |   81 -
 .../cookbook/CombinePerKeyExamplesTest.java     |   91 -
 .../examples/cookbook/DeDupExampleTest.java     |   84 -
 .../examples/cookbook/FilterExamplesTest.java   |   86 -
 .../examples/cookbook/JoinExamplesTest.java     |  115 -
 .../cookbook/MaxPerKeyExamplesTest.java         |   86 -
 .../examples/cookbook/TriggerExampleTest.java   |  140 -
 .../beam/examples/DebuggingWordCountTest.java   |   45 +
 .../org/apache/beam/examples/WordCountTest.java |   86 +
 .../examples/complete/AutoCompleteTest.java     |  182 ++
 .../beam/examples/complete/TfIdfTest.java       |   68 +
 .../complete/TopWikipediaSessionsTest.java      |   63 +
 .../cookbook/BigQueryTornadoesTest.java         |   81 +
 .../cookbook/CombinePerKeyExamplesTest.java     |   91 +
 .../examples/cookbook/DeDupExampleTest.java     |   84 +
 .../examples/cookbook/FilterExamplesTest.java   |   86 +
 .../examples/cookbook/JoinExamplesTest.java     |  115 +
 .../cookbook/MaxPerKeyExamplesTest.java         |   86 +
 .../examples/cookbook/TriggerExampleTest.java   |  138 +
 .../examples/MinimalWordCountJava8.java         |   69 -
 .../examples/complete/game/GameStats.java       |  340 --
 .../examples/complete/game/HourlyTeamScore.java |  194 --
 .../examples/complete/game/LeaderBoard.java     |  238 --
 .../dataflow/examples/complete/game/README.md   |  113 -
 .../examples/complete/game/UserScore.java       |  240 --
 .../complete/game/injector/Injector.java        |  416 ---
 .../complete/game/injector/InjectorUtils.java   |  102 -
 .../injector/RetryHttpInitializerWrapper.java   |  129 -
 .../complete/game/utils/WriteToBigQuery.java    |  135 -
 .../game/utils/WriteWindowedToBigQuery.java     |   77 -
 .../beam/examples/MinimalWordCountJava8.java    |   69 +
 .../beam/examples/complete/game/GameStats.java  |  340 ++
 .../examples/complete/game/HourlyTeamScore.java |  194 ++
 .../examples/complete/game/LeaderBoard.java     |  238 ++
 .../beam/examples/complete/game/README.md       |  113 +
 .../beam/examples/complete/game/UserScore.java  |  240 ++
 .../complete/game/injector/Injector.java        |  416 +++
 .../complete/game/injector/InjectorUtils.java   |  102 +
 .../injector/RetryHttpInitializerWrapper.java   |  129 +
 .../complete/game/utils/WriteToBigQuery.java    |  135 +
 .../game/utils/WriteWindowedToBigQuery.java     |   77 +
 .../examples/MinimalWordCountJava8Test.java     |  104 -
 .../examples/complete/game/GameStatsTest.java   |   77 -
 .../complete/game/HourlyTeamScoreTest.java      |  112 -
 .../examples/complete/game/UserScoreTest.java   |  155 -
 .../examples/MinimalWordCountJava8Test.java     |  104 +
 .../examples/complete/game/GameStatsTest.java   |   77 +
 .../complete/game/HourlyTeamScoreTest.java      |  112 +
 .../examples/complete/game/UserScoreTest.java   |  155 +
 .../BlockingDataflowPipelineOptions.java        |   50 -
 .../sdk/options/CloudDebuggerOptions.java       |   53 -
 .../options/DataflowPipelineDebugOptions.java   |  254 --
 .../sdk/options/DataflowPipelineOptions.java    |  115 -
 .../DataflowPipelineWorkerPoolOptions.java      |  258 --
 .../sdk/options/DataflowProfilingOptions.java   |   48 -
 .../options/DataflowWorkerHarnessOptions.java   |   51 -
 .../options/DataflowWorkerLoggingOptions.java   |  155 -
 .../runners/BlockingDataflowPipelineRunner.java |  186 --
 .../DataflowJobAlreadyExistsException.java      |   35 -
 .../DataflowJobAlreadyUpdatedException.java     |   34 -
 .../runners/DataflowJobCancelledException.java  |   39 -
 .../sdk/runners/DataflowJobException.java       |   41 -
 .../runners/DataflowJobExecutionException.java  |   35 -
 .../runners/DataflowJobUpdatedException.java    |   52 -
 .../dataflow/sdk/runners/DataflowPipeline.java  |   60 -
 .../sdk/runners/DataflowPipelineJob.java        |  394 ---
 .../sdk/runners/DataflowPipelineRegistrar.java  |   59 -
 .../sdk/runners/DataflowPipelineRunner.java     | 3009 ------------------
 .../runners/DataflowPipelineRunnerHooks.java    |   39 -
 .../sdk/runners/DataflowPipelineTranslator.java | 1100 -------
 .../sdk/runners/DataflowServiceException.java   |   33 -
 .../sdk/runners/dataflow/AssignWindows.java     |   90 -
 .../runners/dataflow/BigQueryIOTranslator.java  |  126 -
 .../sdk/runners/dataflow/CustomSources.java     |  119 -
 .../dataflow/DataflowAggregatorTransforms.java  |   80 -
 .../dataflow/DataflowMetricUpdateExtractor.java |  111 -
 .../runners/dataflow/PubsubIOTranslator.java    |  108 -
 .../sdk/runners/dataflow/ReadTranslator.java    |  104 -
 .../sdk/runners/dataflow/package-info.java      |   21 -
 .../testing/TestDataflowPipelineOptions.java    |   27 -
 .../sdk/testing/TestDataflowPipelineRunner.java |  256 --
 .../sdk/util/DataflowPathValidator.java         |   98 -
 .../dataflow/sdk/util/DataflowTransport.java    |  112 -
 .../cloud/dataflow/sdk/util/GcsStager.java      |   54 -
 .../cloud/dataflow/sdk/util/MonitoringUtil.java |  235 --
 .../cloud/dataflow/sdk/util/PackageUtil.java    |  328 --
 .../google/cloud/dataflow/sdk/util/Stager.java  |   30 -
 .../BlockingDataflowPipelineOptions.java        |   50 +
 .../beam/sdk/options/CloudDebuggerOptions.java  |   53 +
 .../options/DataflowPipelineDebugOptions.java   |  254 ++
 .../sdk/options/DataflowPipelineOptions.java    |  115 +
 .../DataflowPipelineWorkerPoolOptions.java      |  258 ++
 .../sdk/options/DataflowProfilingOptions.java   |   48 +
 .../options/DataflowWorkerHarnessOptions.java   |   51 +
 .../options/DataflowWorkerLoggingOptions.java   |  155 +
 .../runners/BlockingDataflowPipelineRunner.java |  186 ++
 .../DataflowJobAlreadyExistsException.java      |   35 +
 .../DataflowJobAlreadyUpdatedException.java     |   34 +
 .../runners/DataflowJobCancelledException.java  |   39 +
 .../beam/sdk/runners/DataflowJobException.java  |   41 +
 .../runners/DataflowJobExecutionException.java  |   35 +
 .../runners/DataflowJobUpdatedException.java    |   51 +
 .../beam/sdk/runners/DataflowPipeline.java      |   60 +
 .../beam/sdk/runners/DataflowPipelineJob.java   |  394 +++
 .../sdk/runners/DataflowPipelineRegistrar.java  |   59 +
 .../sdk/runners/DataflowPipelineRunner.java     | 3009 ++++++++++++++++++
 .../runners/DataflowPipelineRunnerHooks.java    |   39 +
 .../sdk/runners/DataflowPipelineTranslator.java | 1100 +++++++
 .../sdk/runners/DataflowServiceException.java   |   33 +
 .../sdk/runners/dataflow/AssignWindows.java     |   89 +
 .../runners/dataflow/BigQueryIOTranslator.java  |  126 +
 .../sdk/runners/dataflow/CustomSources.java     |  119 +
 .../dataflow/DataflowAggregatorTransforms.java  |   80 +
 .../dataflow/DataflowMetricUpdateExtractor.java |  110 +
 .../runners/dataflow/PubsubIOTranslator.java    |  108 +
 .../sdk/runners/dataflow/ReadTranslator.java    |  104 +
 .../beam/sdk/runners/dataflow/package-info.java |   21 +
 .../testing/TestDataflowPipelineOptions.java    |   27 +
 .../sdk/testing/TestDataflowPipelineRunner.java |  256 ++
 .../beam/sdk/util/DataflowPathValidator.java    |   98 +
 .../apache/beam/sdk/util/DataflowTransport.java |  112 +
 .../org/apache/beam/sdk/util/GcsStager.java     |   54 +
 .../apache/beam/sdk/util/MonitoringUtil.java    |  235 ++
 .../org/apache/beam/sdk/util/PackageUtil.java   |  328 ++
 .../java/org/apache/beam/sdk/util/Stager.java   |   30 +
 .../dataflow/sdk/io/DataflowTextIOTest.java     |  118 -
 .../DataflowPipelineDebugOptionsTest.java       |   41 -
 .../options/DataflowPipelineOptionsTest.java    |   92 -
 .../options/DataflowProfilingOptionsTest.java   |   49 -
 .../DataflowWorkerLoggingOptionsTest.java       |   75 -
 .../BlockingDataflowPipelineRunnerTest.java     |  302 --
 .../sdk/runners/DataflowPipelineJobTest.java    |  605 ----
 .../runners/DataflowPipelineRegistrarTest.java  |   73 -
 .../sdk/runners/DataflowPipelineRunnerTest.java | 1369 --------
 .../sdk/runners/DataflowPipelineTest.java       |   45 -
 .../runners/DataflowPipelineTranslatorTest.java |  890 ------
 .../sdk/runners/dataflow/CustomSourcesTest.java |  274 --
 .../testing/TestDataflowPipelineRunnerTest.java |  377 ---
 .../sdk/transforms/DataflowGroupByKeyTest.java  |  111 -
 .../sdk/transforms/DataflowViewTest.java        |  206 --
 .../sdk/util/DataflowPathValidatorTest.java     |   93 -
 .../dataflow/sdk/util/MonitoringUtilTest.java   |  148 -
 .../dataflow/sdk/util/PackageUtilTest.java      |  483 ---
 .../apache/beam/sdk/io/DataflowTextIOTest.java  |  117 +
 .../DataflowPipelineDebugOptionsTest.java       |   41 +
 .../options/DataflowPipelineOptionsTest.java    |   92 +
 .../options/DataflowProfilingOptionsTest.java   |   49 +
 .../DataflowWorkerLoggingOptionsTest.java       |   75 +
 .../BlockingDataflowPipelineRunnerTest.java     |  302 ++
 .../sdk/runners/DataflowPipelineJobTest.java    |  605 ++++
 .../runners/DataflowPipelineRegistrarTest.java  |   73 +
 .../sdk/runners/DataflowPipelineRunnerTest.java | 1369 ++++++++
 .../beam/sdk/runners/DataflowPipelineTest.java  |   45 +
 .../runners/DataflowPipelineTranslatorTest.java |  890 ++++++
 .../sdk/runners/dataflow/CustomSourcesTest.java |  274 ++
 .../testing/TestDataflowPipelineRunnerTest.java |  377 +++
 .../sdk/transforms/DataflowGroupByKeyTest.java  |  110 +
 .../beam/sdk/transforms/DataflowViewTest.java   |  205 ++
 .../sdk/util/DataflowPathValidatorTest.java     |   92 +
 .../beam/sdk/util/MonitoringUtilTest.java       |  148 +
 .../apache/beam/sdk/util/PackageUtilTest.java   |  483 +++
 .../com/google/cloud/dataflow/sdk/Pipeline.java |  503 ---
 .../cloud/dataflow/sdk/PipelineResult.java      |   96 -
 .../cloud/dataflow/sdk/coders/AtomicCoder.java  |   52 -
 .../cloud/dataflow/sdk/coders/AvroCoder.java    |  715 -----
 .../sdk/coders/BigEndianIntegerCoder.java       |  100 -
 .../dataflow/sdk/coders/BigEndianLongCoder.java |  100 -
 .../dataflow/sdk/coders/ByteArrayCoder.java     |  139 -
 .../cloud/dataflow/sdk/coders/ByteCoder.java    |  112 -
 .../dataflow/sdk/coders/ByteStringCoder.java    |  107 -
 .../sdk/coders/CannotProvideCoderException.java |   96 -
 .../google/cloud/dataflow/sdk/coders/Coder.java |  299 --
 .../dataflow/sdk/coders/CoderException.java     |   37 -
 .../dataflow/sdk/coders/CoderFactories.java     |  275 --
 .../cloud/dataflow/sdk/coders/CoderFactory.java |   44 -
 .../dataflow/sdk/coders/CoderProvider.java      |   34 -
 .../dataflow/sdk/coders/CoderProviders.java     |  165 -
 .../dataflow/sdk/coders/CoderRegistry.java      |  844 -----
 .../dataflow/sdk/coders/CollectionCoder.java    |   74 -
 .../cloud/dataflow/sdk/coders/CustomCoder.java  |  138 -
 .../cloud/dataflow/sdk/coders/DefaultCoder.java |   67 -
 .../dataflow/sdk/coders/DelegateCoder.java      |  165 -
 .../sdk/coders/DeterministicStandardCoder.java  |   39 -
 .../cloud/dataflow/sdk/coders/DoubleCoder.java  |  114 -
 .../dataflow/sdk/coders/DurationCoder.java      |   98 -
 .../cloud/dataflow/sdk/coders/EntityCoder.java  |   87 -
 .../cloud/dataflow/sdk/coders/InstantCoder.java |  114 -
 .../dataflow/sdk/coders/IterableCoder.java      |   79 -
 .../dataflow/sdk/coders/IterableLikeCoder.java  |  279 --
 .../cloud/dataflow/sdk/coders/JAXBCoder.java    |  170 -
 .../cloud/dataflow/sdk/coders/KvCoder.java      |  163 -
 .../cloud/dataflow/sdk/coders/KvCoderBase.java  |   62 -
 .../cloud/dataflow/sdk/coders/ListCoder.java    |   78 -
 .../cloud/dataflow/sdk/coders/MapCoder.java     |  161 -
 .../cloud/dataflow/sdk/coders/MapCoderBase.java |   55 -
 .../dataflow/sdk/coders/NullableCoder.java      |  179 --
 .../cloud/dataflow/sdk/coders/Proto2Coder.java  |  363 ---
 .../dataflow/sdk/coders/SerializableCoder.java  |  184 --
 .../cloud/dataflow/sdk/coders/SetCoder.java     |   95 -
 .../dataflow/sdk/coders/StandardCoder.java      |  230 --
 .../sdk/coders/StringDelegateCoder.java         |   87 -
 .../dataflow/sdk/coders/StringUtf8Coder.java    |  140 -
 .../sdk/coders/StructuralByteArray.java         |   58 -
 .../dataflow/sdk/coders/TableRowJsonCoder.java  |   83 -
 .../sdk/coders/TextualIntegerCoder.java         |   70 -
 .../cloud/dataflow/sdk/coders/VarIntCoder.java  |   98 -
 .../cloud/dataflow/sdk/coders/VarLongCoder.java |   97 -
 .../cloud/dataflow/sdk/coders/VoidCoder.java    |   77 -
 .../cloud/dataflow/sdk/coders/package-info.java |   45 -
 .../sdk/coders/protobuf/ProtoCoder.java         |  406 ---
 .../sdk/coders/protobuf/ProtobufUtil.java       |  172 -
 .../sdk/coders/protobuf/package-info.java       |   24 -
 .../google/cloud/dataflow/sdk/io/AvroIO.java    |  811 -----
 .../cloud/dataflow/sdk/io/AvroSource.java       |  648 ----
 .../cloud/dataflow/sdk/io/BigQueryIO.java       | 1685 ----------
 .../cloud/dataflow/sdk/io/BlockBasedSource.java |  238 --
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  272 --
 .../cloud/dataflow/sdk/io/BoundedSource.java    |  278 --
 .../cloud/dataflow/sdk/io/CompressedSource.java |  414 ---
 .../cloud/dataflow/sdk/io/CountingInput.java    |  224 --
 .../cloud/dataflow/sdk/io/CountingSource.java   |  481 ---
 .../cloud/dataflow/sdk/io/DatastoreIO.java      |  947 ------
 .../cloud/dataflow/sdk/io/FileBasedSink.java    |  865 -----
 .../cloud/dataflow/sdk/io/FileBasedSource.java  |  663 ----
 .../dataflow/sdk/io/OffsetBasedSource.java      |  329 --
 .../cloud/dataflow/sdk/io/PubsubClient.java     |  322 --
 .../cloud/dataflow/sdk/io/PubsubGrpcClient.java |  401 ---
 .../google/cloud/dataflow/sdk/io/PubsubIO.java  | 1048 ------
 .../com/google/cloud/dataflow/sdk/io/Read.java  |  254 --
 .../dataflow/sdk/io/ShardNameTemplate.java      |   76 -
 .../com/google/cloud/dataflow/sdk/io/Sink.java  |  255 --
 .../google/cloud/dataflow/sdk/io/Source.java    |  194 --
 .../google/cloud/dataflow/sdk/io/TextIO.java    |  991 ------
 .../cloud/dataflow/sdk/io/UnboundedSource.java  |  254 --
 .../com/google/cloud/dataflow/sdk/io/Write.java |  217 --
 .../google/cloud/dataflow/sdk/io/XmlSink.java   |  311 --
 .../google/cloud/dataflow/sdk/io/XmlSource.java |  544 ----
 .../dataflow/sdk/io/bigtable/BigtableIO.java    |  989 ------
 .../sdk/io/bigtable/BigtableService.java        |  110 -
 .../sdk/io/bigtable/BigtableServiceImpl.java    |  243 --
 .../dataflow/sdk/io/bigtable/package-info.java  |   23 -
 .../cloud/dataflow/sdk/io/package-info.java     |   38 -
 .../cloud/dataflow/sdk/io/range/ByteKey.java    |  174 -
 .../dataflow/sdk/io/range/ByteKeyRange.java     |  377 ---
 .../sdk/io/range/ByteKeyRangeTracker.java       |  118 -
 .../sdk/io/range/OffsetRangeTracker.java        |  183 --
 .../dataflow/sdk/io/range/RangeTracker.java     |  221 --
 .../dataflow/sdk/io/range/package-info.java     |   24 -
 .../sdk/options/ApplicationNameOptions.java     |   34 -
 .../dataflow/sdk/options/BigQueryOptions.java   |   32 -
 .../cloud/dataflow/sdk/options/Default.java     |  154 -
 .../sdk/options/DefaultValueFactory.java        |   39 -
 .../cloud/dataflow/sdk/options/Description.java |   36 -
 .../sdk/options/DirectPipelineOptions.java      |   76 -
 .../cloud/dataflow/sdk/options/GcpOptions.java  |  292 --
 .../cloud/dataflow/sdk/options/GcsOptions.java  |  114 -
 .../sdk/options/GoogleApiDebugOptions.java      |   88 -
 .../cloud/dataflow/sdk/options/Hidden.java      |   34 -
 .../dataflow/sdk/options/PipelineOptions.java   |  264 --
 .../sdk/options/PipelineOptionsFactory.java     | 1490 ---------
 .../sdk/options/PipelineOptionsRegistrar.java   |   37 -
 .../sdk/options/PipelineOptionsValidator.java   |  103 -
 .../sdk/options/ProxyInvocationHandler.java     |  442 ---
 .../dataflow/sdk/options/PubsubOptions.java     |   36 -
 .../dataflow/sdk/options/StreamingOptions.java  |   31 -
 .../cloud/dataflow/sdk/options/Validation.java  |   47 -
 .../dataflow/sdk/options/package-info.java      |   26 -
 .../google/cloud/dataflow/sdk/package-info.java |   34 -
 .../runners/AggregatorPipelineExtractor.java    |   98 -
 .../runners/AggregatorRetrievalException.java   |   33 -
 .../dataflow/sdk/runners/AggregatorValues.java  |   53 -
 .../dataflow/sdk/runners/DirectPipeline.java    |   56 -
 .../sdk/runners/DirectPipelineRegistrar.java    |   54 -
 .../sdk/runners/DirectPipelineRunner.java       | 1320 --------
 .../dataflow/sdk/runners/PipelineRunner.java    |   77 -
 .../sdk/runners/PipelineRunnerRegistrar.java    |   41 -
 .../sdk/runners/RecordingPipelineVisitor.java   |   55 -
 .../sdk/runners/TransformHierarchy.java         |  105 -
 .../dataflow/sdk/runners/TransformTreeNode.java |  253 --
 .../inprocess/AbstractModelEnforcement.java     |   38 -
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 -
 .../inprocess/BoundedReadEvaluatorFactory.java  |  156 -
 .../sdk/runners/inprocess/BundleFactory.java    |   50 -
 .../CachedThreadPoolExecutorServiceFactory.java |   44 -
 .../dataflow/sdk/runners/inprocess/Clock.java   |   31 -
 .../runners/inprocess/CompletionCallback.java   |   36 -
 .../ConsumerTrackingPipelineVisitor.java        |  175 -
 .../inprocess/EmptyTransformEvaluator.java      |   51 -
 .../EncodabilityEnforcementFactory.java         |   71 -
 .../sdk/runners/inprocess/EvaluatorKey.java     |   57 -
 .../inprocess/ExecutorServiceFactory.java       |   34 -
 .../ExecutorServiceParallelExecutor.java        |  478 ---
 .../inprocess/FlattenEvaluatorFactory.java      |   85 -
 .../runners/inprocess/ForwardingPTransform.java |   56 -
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  273 --
 .../ImmutabilityCheckingBundleFactory.java      |  131 -
 .../ImmutabilityEnforcementFactory.java         |  104 -
 .../inprocess/InMemoryWatermarkManager.java     | 1313 --------
 .../sdk/runners/inprocess/InProcessBundle.java  |  123 -
 .../inprocess/InProcessBundleFactory.java       |  157 -
 .../inprocess/InProcessBundleOutputManager.java |   52 -
 .../sdk/runners/inprocess/InProcessCreate.java  |  235 --
 .../inprocess/InProcessEvaluationContext.java   |  411 ---
 .../inprocess/InProcessExecutionContext.java    |  108 -
 .../runners/inprocess/InProcessExecutor.java    |   48 -
 .../inprocess/InProcessPipelineOptions.java     |  101 -
 .../inprocess/InProcessPipelineRunner.java      |  366 ---
 .../runners/inprocess/InProcessRegistrar.java   |   54 -
 .../inprocess/InProcessSideInputContainer.java  |  232 --
 .../inprocess/InProcessTimerInternals.java      |   85 -
 .../inprocess/InProcessTransformResult.java     |   77 -
 .../inprocess/KeyedPValueTrackingVisitor.java   |   97 -
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 -
 .../inprocess/ModelEnforcementFactory.java      |   30 -
 .../sdk/runners/inprocess/NanosOffsetClock.java |   60 -
 .../inprocess/PTransformOverrideFactory.java    |   33 -
 .../inprocess/ParDoInProcessEvaluator.java      |  120 -
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   92 -
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   89 -
 .../PassthroughTransformEvaluator.java          |   49 -
 .../runners/inprocess/ShardControlledWrite.java |   81 -
 .../sdk/runners/inprocess/StepAndKey.java       |   70 -
 .../runners/inprocess/StepTransformResult.java  |  164 -
 .../inprocess/TextIOShardedWriteFactory.java    |   78 -
 .../runners/inprocess/TransformEvaluator.java   |   47 -
 .../inprocess/TransformEvaluatorFactory.java    |   44 -
 .../inprocess/TransformEvaluatorRegistry.java   |   76 -
 .../runners/inprocess/TransformExecutor.java    |  175 -
 .../inprocess/TransformExecutorService.java     |   36 -
 .../inprocess/TransformExecutorServices.java    |  155 -
 .../UnboundedReadEvaluatorFactory.java          |  178 --
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 -
 .../inprocess/WatermarkCallbackExecutor.java    |  145 -
 .../inprocess/WindowEvaluatorFactory.java       |  131 -
 .../dataflow/sdk/runners/package-info.java      |   34 -
 .../dataflow/sdk/runners/worker/IsmFormat.java  |  948 ------
 .../sdk/runners/worker/package-info.java        |   25 -
 .../dataflow/sdk/testing/CoderProperties.java   |  350 --
 .../cloud/dataflow/sdk/testing/PAssert.java     |  825 -----
 .../dataflow/sdk/testing/RunnableOnService.java |   34 -
 .../sdk/testing/SerializableMatcher.java        |   37 -
 .../sdk/testing/SerializableMatchers.java       | 1181 -------
 .../dataflow/sdk/testing/SourceTestUtils.java   |  676 ----
 .../dataflow/sdk/testing/TestPipeline.java      |  205 --
 .../dataflow/sdk/testing/WindowFnTestUtils.java |  326 --
 .../dataflow/sdk/testing/package-info.java      |   22 -
 .../dataflow/sdk/transforms/Aggregator.java     |   79 -
 .../sdk/transforms/AggregatorRetriever.java     |   37 -
 .../sdk/transforms/AppliedPTransform.java       |  101 -
 .../sdk/transforms/ApproximateQuantiles.java    |  767 -----
 .../sdk/transforms/ApproximateUnique.java       |  420 ---
 .../cloud/dataflow/sdk/transforms/Combine.java  | 2242 -------------
 .../dataflow/sdk/transforms/CombineFnBase.java  |  285 --
 .../dataflow/sdk/transforms/CombineFns.java     | 1010 ------
 .../sdk/transforms/CombineWithContext.java      |  279 --
 .../cloud/dataflow/sdk/transforms/Count.java    |  193 --
 .../cloud/dataflow/sdk/transforms/Create.java   |  427 ---
 .../cloud/dataflow/sdk/transforms/DoFn.java     |  564 ----
 .../dataflow/sdk/transforms/DoFnReflector.java  |  669 ----
 .../dataflow/sdk/transforms/DoFnTester.java     |  496 ---
 .../sdk/transforms/DoFnWithContext.java         |  417 ---
 .../cloud/dataflow/sdk/transforms/Filter.java   |  235 --
 .../sdk/transforms/FlatMapElements.java         |  146 -
 .../cloud/dataflow/sdk/transforms/Flatten.java  |  220 --
 .../dataflow/sdk/transforms/GroupByKey.java     |  276 --
 .../transforms/IntraBundleParallelization.java  |  348 --
 .../cloud/dataflow/sdk/transforms/Keys.java     |   69 -
 .../cloud/dataflow/sdk/transforms/KvSwap.java   |   74 -
 .../dataflow/sdk/transforms/MapElements.java    |  113 -
 .../cloud/dataflow/sdk/transforms/Max.java      |  256 --
 .../cloud/dataflow/sdk/transforms/Mean.java     |  203 --
 .../cloud/dataflow/sdk/transforms/Min.java      |  256 --
 .../dataflow/sdk/transforms/PTransform.java     |  325 --
 .../cloud/dataflow/sdk/transforms/ParDo.java    | 1323 --------
 .../dataflow/sdk/transforms/Partition.java      |  174 -
 .../sdk/transforms/RemoveDuplicates.java        |  159 -
 .../cloud/dataflow/sdk/transforms/Sample.java   |  247 --
 .../sdk/transforms/SerializableComparator.java  |   29 -
 .../sdk/transforms/SerializableFunction.java    |   32 -
 .../dataflow/sdk/transforms/SimpleFunction.java |   55 -
 .../cloud/dataflow/sdk/transforms/Sum.java      |  189 --
 .../cloud/dataflow/sdk/transforms/Top.java      |  560 ----
 .../cloud/dataflow/sdk/transforms/Values.java   |   69 -
 .../cloud/dataflow/sdk/transforms/View.java     |  471 ---
 .../cloud/dataflow/sdk/transforms/WithKeys.java |  141 -
 .../dataflow/sdk/transforms/WithTimestamps.java |  130 -
 .../cloud/dataflow/sdk/transforms/Write.java    |   30 -
 .../sdk/transforms/display/DisplayData.java     |  901 ------
 .../sdk/transforms/display/HasDisplayData.java  |   55 -
 .../sdk/transforms/join/CoGbkResult.java        |  464 ---
 .../sdk/transforms/join/CoGbkResultSchema.java  |  135 -
 .../sdk/transforms/join/CoGroupByKey.java       |  212 --
 .../transforms/join/KeyedPCollectionTuple.java  |  248 --
 .../sdk/transforms/join/RawUnionValue.java      |   52 -
 .../sdk/transforms/join/UnionCoder.java         |  148 -
 .../sdk/transforms/join/package-info.java       |   22 -
 .../dataflow/sdk/transforms/package-info.java   |   44 -
 .../sdk/transforms/windowing/AfterAll.java      |  114 -
 .../windowing/AfterDelayFromFirstElement.java   |  323 --
 .../sdk/transforms/windowing/AfterEach.java     |  133 -
 .../sdk/transforms/windowing/AfterFirst.java    |  117 -
 .../sdk/transforms/windowing/AfterPane.java     |  143 -
 .../windowing/AfterProcessingTime.java          |   96 -
 .../AfterSynchronizedProcessingTime.java        |   76 -
 .../transforms/windowing/AfterWatermark.java    |  396 ---
 .../sdk/transforms/windowing/BoundedWindow.java |   47 -
 .../transforms/windowing/CalendarWindows.java   |  349 --
 .../transforms/windowing/DefaultTrigger.java    |   94 -
 .../sdk/transforms/windowing/FixedWindows.java  |  117 -
 .../sdk/transforms/windowing/GlobalWindow.java  |   69 -
 .../sdk/transforms/windowing/GlobalWindows.java |   64 -
 .../transforms/windowing/IntervalWindow.java    |  202 --
 .../transforms/windowing/InvalidWindows.java    |   88 -
 .../MergeOverlappingIntervalWindows.java        |   87 -
 .../windowing/NonMergingWindowFn.java           |   36 -
 .../transforms/windowing/OrFinallyTrigger.java  |  102 -
 .../sdk/transforms/windowing/OutputTimeFn.java  |  320 --
 .../sdk/transforms/windowing/OutputTimeFns.java |  169 -
 .../sdk/transforms/windowing/PaneInfo.java      |  385 ---
 .../windowing/PartitioningWindowFn.java         |   62 -
 .../sdk/transforms/windowing/Repeatedly.java    |   98 -
 .../sdk/transforms/windowing/Sessions.java      |  113 -
 .../transforms/windowing/SlidingWindows.java    |  215 --
 .../sdk/transforms/windowing/Trigger.java       |  535 ----
 .../transforms/windowing/TriggerBuilder.java    |   29 -
 .../sdk/transforms/windowing/Window.java        |  654 ----
 .../sdk/transforms/windowing/WindowFn.java      |  222 --
 .../sdk/transforms/windowing/package-info.java  |   50 -
 .../dataflow/sdk/util/ActiveWindowSet.java      |  182 --
 .../cloud/dataflow/sdk/util/ApiSurface.java     |  643 ----
 .../dataflow/sdk/util/AppEngineEnvironment.java |   62 -
 .../dataflow/sdk/util/AppliedCombineFn.java     |  132 -
 .../cloud/dataflow/sdk/util/AssignWindows.java  |   46 -
 .../dataflow/sdk/util/AssignWindowsDoFn.java    |   68 -
 ...AttemptAndTimeBoundedExponentialBackOff.java |  169 -
 .../util/AttemptBoundedExponentialBackOff.java  |   84 -
 .../cloud/dataflow/sdk/util/AvroUtils.java      |  346 --
 .../dataflow/sdk/util/BaseExecutionContext.java |  156 -
 .../dataflow/sdk/util/BatchTimerInternals.java  |  139 -
 .../dataflow/sdk/util/BigQueryServices.java     |   61 -
 .../dataflow/sdk/util/BigQueryServicesImpl.java |  196 --
 .../sdk/util/BigQueryTableInserter.java         |  459 ---
 .../sdk/util/BigQueryTableRowIterator.java      |  472 ---
 .../cloud/dataflow/sdk/util/BitSetCoder.java    |   61 -
 .../BufferedElementCountingOutputStream.java    |  187 --
 .../cloud/dataflow/sdk/util/CloudKnownType.java |  139 -
 .../cloud/dataflow/sdk/util/CloudObject.java    |  185 --
 .../cloud/dataflow/sdk/util/CoderUtils.java     |  328 --
 .../sdk/util/CombineContextFactory.java         |  109 -
 .../cloud/dataflow/sdk/util/CombineFnUtil.java  |  205 --
 .../dataflow/sdk/util/CounterAggregator.java    |   97 -
 .../dataflow/sdk/util/CredentialFactory.java    |   30 -
 .../cloud/dataflow/sdk/util/Credentials.java    |  193 --
 .../dataflow/sdk/util/DataflowReleaseInfo.java  |   88 -
 .../sdk/util/DirectModeExecutionContext.java    |  131 -
 .../sdk/util/DirectSideInputReader.java         |   74 -
 .../cloud/dataflow/sdk/util/DoFnInfo.java       |   68 -
 .../cloud/dataflow/sdk/util/DoFnRunner.java     |   62 -
 .../cloud/dataflow/sdk/util/DoFnRunnerBase.java |  559 ----
 .../cloud/dataflow/sdk/util/DoFnRunners.java    |  144 -
 .../dataflow/sdk/util/ExecutableTrigger.java    |  158 -
 .../dataflow/sdk/util/ExecutionContext.java     |  103 -
 .../sdk/util/ExposedByteArrayInputStream.java   |   52 -
 .../sdk/util/ExposedByteArrayOutputStream.java  |  118 -
 .../dataflow/sdk/util/FileIOChannelFactory.java |  136 -
 .../dataflow/sdk/util/FinishedTriggers.java     |   44 -
 .../sdk/util/FinishedTriggersBitSet.java        |   69 -
 .../dataflow/sdk/util/FinishedTriggersSet.java  |   75 -
 .../dataflow/sdk/util/GcpCredentialFactory.java |   46 -
 .../dataflow/sdk/util/GcsIOChannelFactory.java  |   87 -
 .../google/cloud/dataflow/sdk/util/GcsUtil.java |  411 ---
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |  106 -
 .../sdk/util/GroupAlsoByWindowsDoFn.java        |   59 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   99 -
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  246 --
 .../dataflow/sdk/util/IOChannelFactory.java     |  102 -
 .../cloud/dataflow/sdk/util/IOChannelUtils.java |  205 --
 .../sdk/util/IllegalMutationException.java      |   53 -
 .../dataflow/sdk/util/InstanceBuilder.java      |  270 --
 .../util/IntervalBoundedExponentialBackOff.java |   94 -
 .../cloud/dataflow/sdk/util/KeyedWorkItem.java  |   43 -
 .../dataflow/sdk/util/KeyedWorkItemCoder.java   |  122 -
 .../cloud/dataflow/sdk/util/KeyedWorkItems.java |  122 -
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  146 -
 .../dataflow/sdk/util/MapAggregatorValues.java  |   49 -
 .../sdk/util/MergingActiveWindowSet.java        |  575 ----
 .../cloud/dataflow/sdk/util/MimeTypes.java      |   24 -
 .../dataflow/sdk/util/MutationDetector.java     |   32 -
 .../dataflow/sdk/util/MutationDetectors.java    |  182 --
 .../cloud/dataflow/sdk/util/NonEmptyPanes.java  |  150 -
 .../sdk/util/NonMergingActiveWindowSet.java     |   92 -
 .../sdk/util/NoopCredentialFactory.java         |   39 -
 .../dataflow/sdk/util/NoopPathValidator.java    |   49 -
 .../dataflow/sdk/util/NullSideInputReader.java  |   62 -
 .../dataflow/sdk/util/OutputReference.java      |   43 -
 .../sdk/util/PCollectionViewWindow.java         |   68 -
 .../dataflow/sdk/util/PCollectionViews.java     |  427 ---
 .../google/cloud/dataflow/sdk/util/PTuple.java  |  161 -
 .../dataflow/sdk/util/PaneInfoTracker.java      |  153 -
 .../cloud/dataflow/sdk/util/PathValidator.java  |   48 -
 .../sdk/util/PerKeyCombineFnRunner.java         |  149 -
 .../sdk/util/PerKeyCombineFnRunners.java        |  259 --
 .../cloud/dataflow/sdk/util/PropertyNames.java  |  108 -
 .../dataflow/sdk/util/RandomAccessData.java     |  354 ---
 .../cloud/dataflow/sdk/util/ReduceFn.java       |  130 -
 .../sdk/util/ReduceFnContextFactory.java        |  496 ---
 .../cloud/dataflow/sdk/util/ReduceFnRunner.java |  903 ------
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |   47 -
 .../cloud/dataflow/sdk/util/Reshuffle.java      |  147 -
 .../dataflow/sdk/util/ReshuffleTrigger.java     |   63 -
 .../sdk/util/RetryHttpRequestInitializer.java   |  251 --
 .../dataflow/sdk/util/SerializableUtils.java    |  160 -
 .../cloud/dataflow/sdk/util/Serializer.java     |  146 -
 .../sdk/util/ShardingWritableByteChannel.java   |  119 -
 .../dataflow/sdk/util/SideInputReader.java      |   49 -
 .../dataflow/sdk/util/SimpleDoFnRunner.java     |   57 -
 .../cloud/dataflow/sdk/util/StreamUtils.java    |   69 -
 .../cloud/dataflow/sdk/util/StringUtils.java    |  243 --
 .../google/cloud/dataflow/sdk/util/Structs.java |  385 ---
 .../dataflow/sdk/util/SystemDoFnInternal.java   |   39 -
 .../cloud/dataflow/sdk/util/SystemReduceFn.java |  135 -
 .../cloud/dataflow/sdk/util/TestCredential.java |   52 -
 .../cloud/dataflow/sdk/util/TimeDomain.java     |   43 -
 .../cloud/dataflow/sdk/util/TimeUtil.java       |  166 -
 .../cloud/dataflow/sdk/util/TimerInternals.java |  270 --
 .../google/cloud/dataflow/sdk/util/Timers.java  |   60 -
 .../cloud/dataflow/sdk/util/Transport.java      |  151 -
 .../sdk/util/TriggerContextFactory.java         |  512 ---
 .../cloud/dataflow/sdk/util/TriggerRunner.java  |  233 --
 .../dataflow/sdk/util/UnownedInputStream.java   |   77 -
 .../dataflow/sdk/util/UnownedOutputStream.java  |   57 -
 .../sdk/util/UploadIdResponseInterceptor.java   |   62 -
 .../dataflow/sdk/util/UserCodeException.java    |   95 -
 .../dataflow/sdk/util/ValueWithRecordId.java    |  155 -
 .../google/cloud/dataflow/sdk/util/Values.java  |   89 -
 .../google/cloud/dataflow/sdk/util/VarInt.java  |  116 -
 .../cloud/dataflow/sdk/util/WatermarkHold.java  |  476 ---
 .../cloud/dataflow/sdk/util/Weighted.java       |   28 -
 .../cloud/dataflow/sdk/util/WeightedValue.java  |   46 -
 .../cloud/dataflow/sdk/util/WindowTracing.java  |   37 -
 .../cloud/dataflow/sdk/util/WindowedValue.java  |  721 -----
 .../dataflow/sdk/util/WindowingInternals.java   |   83 -
 .../dataflow/sdk/util/WindowingStrategy.java    |  267 --
 .../cloud/dataflow/sdk/util/ZipFiles.java       |  295 --
 .../cloud/dataflow/sdk/util/common/Counter.java | 1104 -------
 .../sdk/util/common/CounterProvider.java        |   27 -
 .../dataflow/sdk/util/common/CounterSet.java    |  178 --
 .../util/common/ElementByteSizeObservable.java  |   42 -
 .../ElementByteSizeObservableIterable.java      |   64 -
 .../ElementByteSizeObservableIterator.java      |   37 -
 .../util/common/ElementByteSizeObserver.java    |   93 -
 .../sdk/util/common/PeekingReiterator.java      |   99 -
 .../sdk/util/common/ReflectHelpers.java         |  211 --
 .../dataflow/sdk/util/common/Reiterable.java    |   28 -
 .../dataflow/sdk/util/common/Reiterator.java    |   40 -
 .../dataflow/sdk/util/common/package-info.java  |   19 -
 .../sdk/util/common/worker/StateSampler.java    |  366 ---
 .../sdk/util/common/worker/package-info.java    |   19 -
 .../cloud/dataflow/sdk/util/gcsfs/GcsPath.java  |  620 ----
 .../dataflow/sdk/util/gcsfs/package-info.java   |   19 -
 .../cloud/dataflow/sdk/util/package-info.java   |   19 -
 .../util/state/AccumulatorCombiningState.java   |   53 -
 .../cloud/dataflow/sdk/util/state/BagState.java |   28 -
 .../dataflow/sdk/util/state/CombiningState.java |   42 -
 .../CopyOnAccessInMemoryStateInternals.java     |  456 ---
 .../sdk/util/state/InMemoryStateInternals.java  |  416 ---
 .../sdk/util/state/MergingStateAccessor.java    |   41 -
 .../dataflow/sdk/util/state/ReadableState.java  |   55 -
 .../cloud/dataflow/sdk/util/state/State.java    |   32 -
 .../dataflow/sdk/util/state/StateAccessor.java  |   37 -
 .../dataflow/sdk/util/state/StateContext.java   |   43 -
 .../dataflow/sdk/util/state/StateContexts.java  |  109 -
 .../dataflow/sdk/util/state/StateInternals.java |   57 -
 .../dataflow/sdk/util/state/StateMerging.java   |  256 --
 .../dataflow/sdk/util/state/StateNamespace.java |   56 -
 .../sdk/util/state/StateNamespaceForTest.java   |   65 -
 .../sdk/util/state/StateNamespaces.java         |  279 --
 .../dataflow/sdk/util/state/StateTable.java     |   91 -
 .../cloud/dataflow/sdk/util/state/StateTag.java |   98 -
 .../dataflow/sdk/util/state/StateTags.java      |  581 ----
 .../dataflow/sdk/util/state/ValueState.java     |   37 -
 .../sdk/util/state/WatermarkHoldState.java      |   44 -
 .../google/cloud/dataflow/sdk/values/KV.java    |  131 -
 .../cloud/dataflow/sdk/values/PBegin.java       |   88 -
 .../cloud/dataflow/sdk/values/PCollection.java  |  251 --
 .../dataflow/sdk/values/PCollectionList.java    |  239 --
 .../dataflow/sdk/values/PCollectionTuple.java   |  265 --
 .../dataflow/sdk/values/PCollectionView.java    |   65 -
 .../google/cloud/dataflow/sdk/values/PDone.java |   48 -
 .../cloud/dataflow/sdk/values/PInput.java       |   57 -
 .../cloud/dataflow/sdk/values/POutput.java      |   77 -
 .../dataflow/sdk/values/POutputValueBase.java   |  103 -
 .../cloud/dataflow/sdk/values/PValue.java       |   39 -
 .../cloud/dataflow/sdk/values/PValueBase.java   |  156 -
 .../dataflow/sdk/values/TimestampedValue.java   |  156 -
 .../cloud/dataflow/sdk/values/TupleTag.java     |  197 --
 .../cloud/dataflow/sdk/values/TupleTagList.java |  149 -
 .../dataflow/sdk/values/TypeDescriptor.java     |  352 --
 .../cloud/dataflow/sdk/values/TypedPValue.java  |  198 --
 .../cloud/dataflow/sdk/values/package-info.java |   53 -
 .../main/java/org/apache/beam/sdk/Pipeline.java |  503 +++
 .../org/apache/beam/sdk/PipelineResult.java     |   96 +
 .../org/apache/beam/sdk/coders/AtomicCoder.java |   52 +
 .../org/apache/beam/sdk/coders/AvroCoder.java   |  715 +++++
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |  100 +
 .../beam/sdk/coders/BigEndianLongCoder.java     |  100 +
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |  139 +
 .../org/apache/beam/sdk/coders/ByteCoder.java   |  112 +
 .../apache/beam/sdk/coders/ByteStringCoder.java |  107 +
 .../sdk/coders/CannotProvideCoderException.java |   96 +
 .../java/org/apache/beam/sdk/coders/Coder.java  |  299 ++
 .../apache/beam/sdk/coders/CoderException.java  |   37 +
 .../apache/beam/sdk/coders/CoderFactories.java  |  275 ++
 .../apache/beam/sdk/coders/CoderFactory.java    |   44 +
 .../apache/beam/sdk/coders/CoderProvider.java   |   34 +
 .../apache/beam/sdk/coders/CoderProviders.java  |  165 +
 .../apache/beam/sdk/coders/CoderRegistry.java   |  844 +++++
 .../apache/beam/sdk/coders/CollectionCoder.java |   74 +
 .../org/apache/beam/sdk/coders/CustomCoder.java |  138 +
 .../apache/beam/sdk/coders/DefaultCoder.java    |   67 +
 .../apache/beam/sdk/coders/DelegateCoder.java   |  165 +
 .../sdk/coders/DeterministicStandardCoder.java  |   39 +
 .../org/apache/beam/sdk/coders/DoubleCoder.java |  114 +
 .../apache/beam/sdk/coders/DurationCoder.java   |   98 +
 .../org/apache/beam/sdk/coders/EntityCoder.java |   87 +
 .../apache/beam/sdk/coders/InstantCoder.java    |  114 +
 .../apache/beam/sdk/coders/IterableCoder.java   |   79 +
 .../beam/sdk/coders/IterableLikeCoder.java      |  279 ++
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |  170 +
 .../org/apache/beam/sdk/coders/KvCoder.java     |  163 +
 .../org/apache/beam/sdk/coders/KvCoderBase.java |   62 +
 .../org/apache/beam/sdk/coders/ListCoder.java   |   78 +
 .../org/apache/beam/sdk/coders/MapCoder.java    |  161 +
 .../apache/beam/sdk/coders/MapCoderBase.java    |   55 +
 .../apache/beam/sdk/coders/NullableCoder.java   |  179 ++
 .../org/apache/beam/sdk/coders/Proto2Coder.java |  363 +++
 .../beam/sdk/coders/SerializableCoder.java      |  184 ++
 .../org/apache/beam/sdk/coders/SetCoder.java    |   95 +
 .../apache/beam/sdk/coders/StandardCoder.java   |  230 ++
 .../beam/sdk/coders/StringDelegateCoder.java    |   87 +
 .../apache/beam/sdk/coders/StringUtf8Coder.java |  140 +
 .../beam/sdk/coders/StructuralByteArray.java    |   58 +
 .../beam/sdk/coders/TableRowJsonCoder.java      |   83 +
 .../beam/sdk/coders/TextualIntegerCoder.java    |   70 +
 .../org/apache/beam/sdk/coders/VarIntCoder.java |   98 +
 .../apache/beam/sdk/coders/VarLongCoder.java    |   97 +
 .../org/apache/beam/sdk/coders/VoidCoder.java   |   77 +
 .../apache/beam/sdk/coders/package-info.java    |   45 +
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |  406 +++
 .../beam/sdk/coders/protobuf/ProtobufUtil.java  |  172 +
 .../beam/sdk/coders/protobuf/package-info.java  |   24 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  811 +++++
 .../java/org/apache/beam/sdk/io/AvroSource.java |  648 ++++
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 1685 ++++++++++
 .../apache/beam/sdk/io/BlockBasedSource.java    |  238 ++
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  272 ++
 .../org/apache/beam/sdk/io/BoundedSource.java   |  278 ++
 .../apache/beam/sdk/io/CompressedSource.java    |  414 +++
 .../org/apache/beam/sdk/io/CountingInput.java   |  224 ++
 .../org/apache/beam/sdk/io/CountingSource.java  |  481 +++
 .../org/apache/beam/sdk/io/DatastoreIO.java     |  947 ++++++
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  865 +++++
 .../org/apache/beam/sdk/io/FileBasedSource.java |  663 ++++
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  329 ++
 .../org/apache/beam/sdk/io/PubsubClient.java    |  322 ++
 .../apache/beam/sdk/io/PubsubGrpcClient.java    |  401 +++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1048 ++++++
 .../main/java/org/apache/beam/sdk/io/Read.java  |  254 ++
 .../apache/beam/sdk/io/ShardNameTemplate.java   |   76 +
 .../main/java/org/apache/beam/sdk/io/Sink.java  |  255 ++
 .../java/org/apache/beam/sdk/io/Source.java     |  194 ++
 .../java/org/apache/beam/sdk/io/TextIO.java     |  991 ++++++
 .../org/apache/beam/sdk/io/UnboundedSource.java |  254 ++
 .../main/java/org/apache/beam/sdk/io/Write.java |  217 ++
 .../java/org/apache/beam/sdk/io/XmlSink.java    |  311 ++
 .../java/org/apache/beam/sdk/io/XmlSource.java  |  544 ++++
 .../apache/beam/sdk/io/bigtable/BigtableIO.java |  989 ++++++
 .../beam/sdk/io/bigtable/BigtableService.java   |  110 +
 .../sdk/io/bigtable/BigtableServiceImpl.java    |  243 ++
 .../beam/sdk/io/bigtable/package-info.java      |   23 +
 .../org/apache/beam/sdk/io/package-info.java    |   38 +
 .../org/apache/beam/sdk/io/range/ByteKey.java   |  174 +
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |  377 +++
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |  118 +
 .../beam/sdk/io/range/OffsetRangeTracker.java   |  183 ++
 .../apache/beam/sdk/io/range/RangeTracker.java  |  221 ++
 .../apache/beam/sdk/io/range/package-info.java  |   24 +
 .../sdk/options/ApplicationNameOptions.java     |   34 +
 .../beam/sdk/options/BigQueryOptions.java       |   32 +
 .../org/apache/beam/sdk/options/Default.java    |  154 +
 .../beam/sdk/options/DefaultValueFactory.java   |   39 +
 .../apache/beam/sdk/options/Description.java    |   36 +
 .../beam/sdk/options/DirectPipelineOptions.java |   76 +
 .../org/apache/beam/sdk/options/GcpOptions.java |  292 ++
 .../org/apache/beam/sdk/options/GcsOptions.java |  114 +
 .../beam/sdk/options/GoogleApiDebugOptions.java |   88 +
 .../org/apache/beam/sdk/options/Hidden.java     |   34 +
 .../beam/sdk/options/PipelineOptions.java       |  264 ++
 .../sdk/options/PipelineOptionsFactory.java     | 1490 +++++++++
 .../sdk/options/PipelineOptionsRegistrar.java   |   37 +
 .../sdk/options/PipelineOptionsValidator.java   |  103 +
 .../sdk/options/ProxyInvocationHandler.java     |  442 +++
 .../apache/beam/sdk/options/PubsubOptions.java  |   36 +
 .../beam/sdk/options/StreamingOptions.java      |   31 +
 .../org/apache/beam/sdk/options/Validation.java |   47 +
 .../apache/beam/sdk/options/package-info.java   |   26 +
 .../java/org/apache/beam/sdk/package-info.java  |   34 +
 .../runners/AggregatorPipelineExtractor.java    |   98 +
 .../runners/AggregatorRetrievalException.java   |   33 +
 .../beam/sdk/runners/AggregatorValues.java      |   52 +
 .../apache/beam/sdk/runners/DirectPipeline.java |   56 +
 .../sdk/runners/DirectPipelineRegistrar.java    |   54 +
 .../beam/sdk/runners/DirectPipelineRunner.java  | 1320 ++++++++
 .../apache/beam/sdk/runners/PipelineRunner.java |   77 +
 .../sdk/runners/PipelineRunnerRegistrar.java    |   41 +
 .../sdk/runners/RecordingPipelineVisitor.java   |   55 +
 .../beam/sdk/runners/TransformHierarchy.java    |  105 +
 .../beam/sdk/runners/TransformTreeNode.java     |  253 ++
 .../inprocess/AbstractModelEnforcement.java     |   38 +
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 +
 .../inprocess/BoundedReadEvaluatorFactory.java  |  156 +
 .../sdk/runners/inprocess/BundleFactory.java    |   49 +
 .../CachedThreadPoolExecutorServiceFactory.java |   44 +
 .../beam/sdk/runners/inprocess/Clock.java       |   30 +
 .../runners/inprocess/CompletionCallback.java   |   36 +
 .../ConsumerTrackingPipelineVisitor.java        |  173 +
 .../inprocess/EmptyTransformEvaluator.java      |   50 +
 .../EncodabilityEnforcementFactory.java         |   71 +
 .../sdk/runners/inprocess/EvaluatorKey.java     |   55 +
 .../inprocess/ExecutorServiceFactory.java       |   33 +
 .../ExecutorServiceParallelExecutor.java        |  477 +++
 .../inprocess/FlattenEvaluatorFactory.java      |   85 +
 .../runners/inprocess/ForwardingPTransform.java |   56 +
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  273 ++
 .../ImmutabilityCheckingBundleFactory.java      |  131 +
 .../ImmutabilityEnforcementFactory.java         |  104 +
 .../inprocess/InMemoryWatermarkManager.java     | 1313 ++++++++
 .../sdk/runners/inprocess/InProcessBundle.java  |  123 +
 .../inprocess/InProcessBundleFactory.java       |  156 +
 .../inprocess/InProcessBundleOutputManager.java |   51 +
 .../sdk/runners/inprocess/InProcessCreate.java  |  235 ++
 .../inprocess/InProcessEvaluationContext.java   |  411 +++
 .../inprocess/InProcessExecutionContext.java    |  107 +
 .../runners/inprocess/InProcessExecutor.java    |   48 +
 .../inprocess/InProcessPipelineOptions.java     |  101 +
 .../inprocess/InProcessPipelineRunner.java      |  366 +++
 .../runners/inprocess/InProcessRegistrar.java   |   54 +
 .../inprocess/InProcessSideInputContainer.java  |  232 ++
 .../inprocess/InProcessTimerInternals.java      |   84 +
 .../inprocess/InProcessTransformResult.java     |   77 +
 .../inprocess/KeyedPValueTrackingVisitor.java   |   96 +
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 +
 .../inprocess/ModelEnforcementFactory.java      |   30 +
 .../sdk/runners/inprocess/NanosOffsetClock.java |   59 +
 .../inprocess/PTransformOverrideFactory.java    |   33 +
 .../inprocess/ParDoInProcessEvaluator.java      |  120 +
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   92 +
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   89 +
 .../PassthroughTransformEvaluator.java          |   49 +
 .../runners/inprocess/ShardControlledWrite.java |   81 +
 .../beam/sdk/runners/inprocess/StepAndKey.java  |   70 +
 .../runners/inprocess/StepTransformResult.java  |  164 +
 .../inprocess/TextIOShardedWriteFactory.java    |   78 +
 .../runners/inprocess/TransformEvaluator.java   |   46 +
 .../inprocess/TransformEvaluatorFactory.java    |   44 +
 .../inprocess/TransformEvaluatorRegistry.java   |   76 +
 .../runners/inprocess/TransformExecutor.java    |  175 +
 .../inprocess/TransformExecutorService.java     |   35 +
 .../inprocess/TransformExecutorServices.java    |  154 +
 .../UnboundedReadEvaluatorFactory.java          |  177 ++
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 +
 .../inprocess/WatermarkCallbackExecutor.java    |  145 +
 .../inprocess/WindowEvaluatorFactory.java       |  131 +
 .../apache/beam/sdk/runners/package-info.java   |   34 +
 .../beam/sdk/runners/worker/IsmFormat.java      |  948 ++++++
 .../beam/sdk/runners/worker/package-info.java   |   25 +
 .../beam/sdk/testing/CoderProperties.java       |  350 ++
 .../org/apache/beam/sdk/testing/PAssert.java    |  825 +++++
 .../beam/sdk/testing/RunnableOnService.java     |   34 +
 .../beam/sdk/testing/SerializableMatcher.java   |   36 +
 .../beam/sdk/testing/SerializableMatchers.java  | 1181 +++++++
 .../beam/sdk/testing/SourceTestUtils.java       |  676 ++++
 .../apache/beam/sdk/testing/TestPipeline.java   |  205 ++
 .../beam/sdk/testing/WindowFnTestUtils.java     |  326 ++
 .../apache/beam/sdk/testing/package-info.java   |   22 +
 .../apache/beam/sdk/transforms/Aggregator.java  |   79 +
 .../sdk/transforms/AggregatorRetriever.java     |   36 +
 .../beam/sdk/transforms/AppliedPTransform.java  |  101 +
 .../sdk/transforms/ApproximateQuantiles.java    |  767 +++++
 .../beam/sdk/transforms/ApproximateUnique.java  |  420 +++
 .../org/apache/beam/sdk/transforms/Combine.java | 2242 +++++++++++++
 .../beam/sdk/transforms/CombineFnBase.java      |  285 ++
 .../apache/beam/sdk/transforms/CombineFns.java  | 1010 ++++++
 .../beam/sdk/transforms/CombineWithContext.java |  279 ++
 .../org/apache/beam/sdk/transforms/Count.java   |  193 ++
 .../org/apache/beam/sdk/transforms/Create.java  |  427 +++
 .../org/apache/beam/sdk/transforms/DoFn.java    |  564 ++++
 .../beam/sdk/transforms/DoFnReflector.java      |  669 ++++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  496 +++
 .../beam/sdk/transforms/DoFnWithContext.java    |  417 +++
 .../org/apache/beam/sdk/transforms/Filter.java  |  235 ++
 .../beam/sdk/transforms/FlatMapElements.java    |  146 +
 .../org/apache/beam/sdk/transforms/Flatten.java |  220 ++
 .../apache/beam/sdk/transforms/GroupByKey.java  |  276 ++
 .../transforms/IntraBundleParallelization.java  |  348 ++
 .../org/apache/beam/sdk/transforms/Keys.java    |   69 +
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   74 +
 .../apache/beam/sdk/transforms/MapElements.java |  113 +
 .../org/apache/beam/sdk/transforms/Max.java     |  256 ++
 .../org/apache/beam/sdk/transforms/Mean.java    |  203 ++
 .../org/apache/beam/sdk/transforms/Min.java     |  256 ++
 .../apache/beam/sdk/transforms/PTransform.java  |  325 ++
 .../org/apache/beam/sdk/transforms/ParDo.java   | 1323 ++++++++
 .../apache/beam/sdk/transforms/Partition.java   |  174 +
 .../beam/sdk/transforms/RemoveDuplicates.java   |  159 +
 .../org/apache/beam/sdk/transforms/Sample.java  |  247 ++
 .../sdk/transforms/SerializableComparator.java  |   29 +
 .../sdk/transforms/SerializableFunction.java    |   32 +
 .../beam/sdk/transforms/SimpleFunction.java     |   55 +
 .../org/apache/beam/sdk/transforms/Sum.java     |  189 ++
 .../org/apache/beam/sdk/transforms/Top.java     |  560 ++++
 .../org/apache/beam/sdk/transforms/Values.java  |   69 +
 .../org/apache/beam/sdk/transforms/View.java    |  471 +++
 .../apache/beam/sdk/transforms/WithKeys.java    |  141 +
 .../beam/sdk/transforms/WithTimestamps.java     |  130 +
 .../org/apache/beam/sdk/transforms/Write.java   |   29 +
 .../sdk/transforms/display/DisplayData.java     |  901 ++++++
 .../sdk/transforms/display/HasDisplayData.java  |   55 +
 .../beam/sdk/transforms/join/CoGbkResult.java   |  464 +++
 .../sdk/transforms/join/CoGbkResultSchema.java  |  135 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |  212 ++
 .../transforms/join/KeyedPCollectionTuple.java  |  248 ++
 .../beam/sdk/transforms/join/RawUnionValue.java |   51 +
 .../beam/sdk/transforms/join/UnionCoder.java    |  148 +
 .../beam/sdk/transforms/join/package-info.java  |   22 +
 .../beam/sdk/transforms/package-info.java       |   43 +
 .../beam/sdk/transforms/windowing/AfterAll.java |  114 +
 .../windowing/AfterDelayFromFirstElement.java   |  323 ++
 .../sdk/transforms/windowing/AfterEach.java     |  133 +
 .../sdk/transforms/windowing/AfterFirst.java    |  117 +
 .../sdk/transforms/windowing/AfterPane.java     |  143 +
 .../windowing/AfterProcessingTime.java          |   96 +
 .../AfterSynchronizedProcessingTime.java        |   76 +
 .../transforms/windowing/AfterWatermark.java    |  396 +++
 .../sdk/transforms/windowing/BoundedWindow.java |   47 +
 .../transforms/windowing/CalendarWindows.java   |  349 ++
 .../transforms/windowing/DefaultTrigger.java    |   94 +
 .../sdk/transforms/windowing/FixedWindows.java  |  117 +
 .../sdk/transforms/windowing/GlobalWindow.java  |   69 +
 .../sdk/transforms/windowing/GlobalWindows.java |   64 +
 .../transforms/windowing/IntervalWindow.java    |  202 ++
 .../transforms/windowing/InvalidWindows.java    |   88 +
 .../MergeOverlappingIntervalWindows.java        |   87 +
 .../windowing/NonMergingWindowFn.java           |   36 +
 .../transforms/windowing/OrFinallyTrigger.java  |  102 +
 .../sdk/transforms/windowing/OutputTimeFn.java  |  320 ++
 .../sdk/transforms/windowing/OutputTimeFns.java |  169 +
 .../beam/sdk/transforms/windowing/PaneInfo.java |  385 +++
 .../windowing/PartitioningWindowFn.java         |   62 +
 .../sdk/transforms/windowing/Repeatedly.java    |   98 +
 .../beam/sdk/transforms/windowing/Sessions.java |  113 +
 .../transforms/windowing/SlidingWindows.java    |  215 ++
 .../beam/sdk/transforms/windowing/Trigger.java  |  535 ++++
 .../transforms/windowing/TriggerBuilder.java    |   29 +
 .../beam/sdk/transforms/windowing/Window.java   |  654 ++++
 .../beam/sdk/transforms/windowing/WindowFn.java |  222 ++
 .../sdk/transforms/windowing/package-info.java  |   50 +
 .../apache/beam/sdk/util/ActiveWindowSet.java   |  182 ++
 .../org/apache/beam/sdk/util/ApiSurface.java    |  643 ++++
 .../beam/sdk/util/AppEngineEnvironment.java     |   62 +
 .../apache/beam/sdk/util/AppliedCombineFn.java  |  132 +
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 +
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   68 +
 ...AttemptAndTimeBoundedExponentialBackOff.java |  169 +
 .../util/AttemptBoundedExponentialBackOff.java  |   84 +
 .../org/apache/beam/sdk/util/AvroUtils.java     |  345 ++
 .../beam/sdk/util/BaseExecutionContext.java     |  156 +
 .../beam/sdk/util/BatchTimerInternals.java      |  139 +
 .../apache/beam/sdk/util/BigQueryServices.java  |   61 +
 .../beam/sdk/util/BigQueryServicesImpl.java     |  196 ++
 .../beam/sdk/util/BigQueryTableInserter.java    |  459 +++
 .../beam/sdk/util/BigQueryTableRowIterator.java |  472 +++
 .../org/apache/beam/sdk/util/BitSetCoder.java   |   61 +
 .../BufferedElementCountingOutputStream.java    |  186 ++
 .../apache/beam/sdk/util/CloudKnownType.java    |  139 +
 .../org/apache/beam/sdk/util/CloudObject.java   |  185 ++
 .../org/apache/beam/sdk/util/CoderUtils.java    |  328 ++
 .../beam/sdk/util/CombineContextFactory.java    |  109 +
 .../org/apache/beam/sdk/util/CombineFnUtil.java |  205 ++
 .../apache/beam/sdk/util/CounterAggregator.java |   97 +
 .../apache/beam/sdk/util/CredentialFactory.java |   30 +
 .../org/apache/beam/sdk/util/Credentials.java   |  193 ++
 .../beam/sdk/util/DataflowReleaseInfo.java      |   88 +
 .../sdk/util/DirectModeExecutionContext.java    |  131 +
 .../beam/sdk/util/DirectSideInputReader.java    |   74 +
 .../java/org/apache/beam/sdk/util/DoFnInfo.java |   68 +
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   62 +
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  559 ++++
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  144 +
 .../apache/beam/sdk/util/ExecutableTrigger.java |  158 +
 .../apache/beam/sdk/util/ExecutionContext.java  |  103 +
 .../sdk/util/ExposedByteArrayInputStream.java   |   52 +
 .../sdk/util/ExposedByteArrayOutputStream.java  |  118 +
 .../beam/sdk/util/FileIOChannelFactory.java     |  136 +
 .../apache/beam/sdk/util/FinishedTriggers.java  |   44 +
 .../beam/sdk/util/FinishedTriggersBitSet.java   |   67 +
 .../beam/sdk/util/FinishedTriggersSet.java      |   73 +
 .../beam/sdk/util/GcpCredentialFactory.java     |   46 +
 .../beam/sdk/util/GcsIOChannelFactory.java      |   87 +
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  411 +++
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |  105 +
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   59 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   99 +
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  246 ++
 .../apache/beam/sdk/util/IOChannelFactory.java  |  102 +
 .../apache/beam/sdk/util/IOChannelUtils.java    |  205 ++
 .../beam/sdk/util/IllegalMutationException.java |   53 +
 .../apache/beam/sdk/util/InstanceBuilder.java   |  270 ++
 .../util/IntervalBoundedExponentialBackOff.java |   94 +
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |   43 +
 .../beam/sdk/util/KeyedWorkItemCoder.java       |  122 +
 .../apache/beam/sdk/util/KeyedWorkItems.java    |  122 +
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  146 +
 .../beam/sdk/util/MapAggregatorValues.java      |   49 +
 .../beam/sdk/util/MergingActiveWindowSet.java   |  575 ++++
 .../org/apache/beam/sdk/util/MimeTypes.java     |   24 +
 .../apache/beam/sdk/util/MutationDetector.java  |   32 +
 .../apache/beam/sdk/util/MutationDetectors.java |  181 ++
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 +
 .../sdk/util/NonMergingActiveWindowSet.java     |   92 +
 .../beam/sdk/util/NoopCredentialFactory.java    |   39 +
 .../apache/beam/sdk/util/NoopPathValidator.java |   49 +
 .../beam/sdk/util/NullSideInputReader.java      |   62 +
 .../apache/beam/sdk/util/OutputReference.java   |   43 +
 .../beam/sdk/util/PCollectionViewWindow.java    |   68 +
 .../apache/beam/sdk/util/PCollectionViews.java  |  427 +++
 .../java/org/apache/beam/sdk/util/PTuple.java   |  161 +
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  153 +
 .../org/apache/beam/sdk/util/PathValidator.java |   48 +
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  149 +
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |  259 ++
 .../org/apache/beam/sdk/util/PropertyNames.java |  108 +
 .../apache/beam/sdk/util/RandomAccessData.java  |  354 +++
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  130 +
 .../beam/sdk/util/ReduceFnContextFactory.java   |  496 +++
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  903 ++++++
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |   47 +
 .../org/apache/beam/sdk/util/Reshuffle.java     |  147 +
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |   63 +
 .../sdk/util/RetryHttpRequestInitializer.java   |  251 ++
 .../apache/beam/sdk/util/SerializableUtils.java |  160 +
 .../org/apache/beam/sdk/util/Serializer.java    |  146 +
 .../sdk/util/ShardingWritableByteChannel.java   |  119 +
 .../apache/beam/sdk/util/SideInputReader.java   |   48 +
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   56 +
 .../org/apache/beam/sdk/util/StreamUtils.java   |   69 +
 .../org/apache/beam/sdk/util/StringUtils.java   |  243 ++
 .../java/org/apache/beam/sdk/util/Structs.java  |  385 +++
 .../beam/sdk/util/SystemDoFnInternal.java       |   39 +
 .../apache/beam/sdk/util/SystemReduceFn.java    |  135 +
 .../apache/beam/sdk/util/TestCredential.java    |   52 +
 .../org/apache/beam/sdk/util/TimeDomain.java    |   43 +
 .../java/org/apache/beam/sdk/util/TimeUtil.java |  166 +
 .../apache/beam/sdk/util/TimerInternals.java    |  270 ++
 .../java/org/apache/beam/sdk/util/Timers.java   |   60 +
 .../org/apache/beam/sdk/util/Transport.java     |  151 +
 .../beam/sdk/util/TriggerContextFactory.java    |  512 +++
 .../org/apache/beam/sdk/util/TriggerRunner.java |  233 ++
 .../beam/sdk/util/UnownedInputStream.java       |   76 +
 .../beam/sdk/util/UnownedOutputStream.java      |   56 +
 .../sdk/util/UploadIdResponseInterceptor.java   |   62 +
 .../apache/beam/sdk/util/UserCodeException.java |   95 +
 .../apache/beam/sdk/util/ValueWithRecordId.java |  155 +
 .../java/org/apache/beam/sdk/util/Values.java   |   89 +
 .../java/org/apache/beam/sdk/util/VarInt.java   |  116 +
 .../org/apache/beam/sdk/util/WatermarkHold.java |  476 +++
 .../java/org/apache/beam/sdk/util/Weighted.java |   28 +
 .../org/apache/beam/sdk/util/WeightedValue.java |   46 +
 .../org/apache/beam/sdk/util/WindowTracing.java |   37 +
 .../org/apache/beam/sdk/util/WindowedValue.java |  721 +++++
 .../beam/sdk/util/WindowingInternals.java       |   83 +
 .../apache/beam/sdk/util/WindowingStrategy.java |  267 ++
 .../java/org/apache/beam/sdk/util/ZipFiles.java |  295 ++
 .../apache/beam/sdk/util/common/Counter.java    | 1104 +++++++
 .../beam/sdk/util/common/CounterProvider.java   |   27 +
 .../apache/beam/sdk/util/common/CounterSet.java |  178 ++
 .../util/common/ElementByteSizeObservable.java  |   42 +
 .../ElementByteSizeObservableIterable.java      |   64 +
 .../ElementByteSizeObservableIterator.java      |   37 +
 .../util/common/ElementByteSizeObserver.java    |   93 +
 .../beam/sdk/util/common/PeekingReiterator.java |   99 +
 .../beam/sdk/util/common/ReflectHelpers.java    |  211 ++
 .../apache/beam/sdk/util/common/Reiterable.java |   28 +
 .../apache/beam/sdk/util/common/Reiterator.java |   40 +
 .../beam/sdk/util/common/package-info.java      |   19 +
 .../sdk/util/common/worker/StateSampler.java    |  366 +++
 .../sdk/util/common/worker/package-info.java    |   19 +
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java |  620 ++++
 .../beam/sdk/util/gcsfs/package-info.java       |   19 +
 .../org/apache/beam/sdk/util/package-info.java  |   19 +
 .../util/state/AccumulatorCombiningState.java   |   53 +
 .../apache/beam/sdk/util/state/BagState.java    |   28 +
 .../beam/sdk/util/state/CombiningState.java     |   42 +
 .../CopyOnAccessInMemoryStateInternals.java     |  456 +++
 .../sdk/util/state/InMemoryStateInternals.java  |  416 +++
 .../sdk/util/state/MergingStateAccessor.java    |   41 +
 .../beam/sdk/util/state/ReadableState.java      |   55 +
 .../org/apache/beam/sdk/util/state/State.java   |   32 +
 .../beam/sdk/util/state/StateAccessor.java      |   37 +
 .../beam/sdk/util/state/StateContext.java       |   43 +
 .../beam/sdk/util/state/StateContexts.java      |  109 +
 .../beam/sdk/util/state/StateInternals.java     |   57 +
 .../beam/sdk/util/state/StateMerging.java       |  256 ++
 .../beam/sdk/util/state/StateNamespace.java     |   56 +
 .../sdk/util/state/StateNamespaceForTest.java   |   65 +
 .../beam/sdk/util/state/StateNamespaces.java    |  279 ++
 .../apache/beam/sdk/util/state/StateTable.java  |   91 +
 .../apache/beam/sdk/util/state/StateTag.java    |   98 +
 .../apache/beam/sdk/util/state/StateTags.java   |  581 ++++
 .../apache/beam/sdk/util/state/ValueState.java  |   37 +
 .../beam/sdk/util/state/WatermarkHoldState.java |   44 +
 .../java/org/apache/beam/sdk/values/KV.java     |  131 +
 .../java/org/apache/beam/sdk/values/PBegin.java |   88 +
 .../org/apache/beam/sdk/values/PCollection.java |  251 ++
 .../apache/beam/sdk/values/PCollectionList.java |  239 ++
 .../beam/sdk/values/PCollectionTuple.java       |  265 ++
 .../apache/beam/sdk/values/PCollectionView.java |   65 +
 .../java/org/apache/beam/sdk/values/PDone.java  |   48 +
 .../java/org/apache/beam/sdk/values/PInput.java |   57 +
 .../org/apache/beam/sdk/values/POutput.java     |   77 +
 .../beam/sdk/values/POutputValueBase.java       |  103 +
 .../java/org/apache/beam/sdk/values/PValue.java |   39 +
 .../org/apache/beam/sdk/values/PValueBase.java  |  156 +
 .../beam/sdk/values/TimestampedValue.java       |  156 +
 .../org/apache/beam/sdk/values/TupleTag.java    |  197 ++
 .../apache/beam/sdk/values/TupleTagList.java    |  149 +
 .../apache/beam/sdk/values/TypeDescriptor.java  |  352 ++
 .../org/apache/beam/sdk/values/TypedPValue.java |  198 ++
 .../apache/beam/sdk/values/package-info.java    |   53 +
 .../cloud/dataflow/sdk/DataflowMatchers.java    |   66 -
 .../google/cloud/dataflow/sdk/PipelineTest.java |  297 --
 .../google/cloud/dataflow/sdk/TestUtils.java    |  214 --
 .../cloud/dataflow/sdk/WindowMatchers.java      |  138 -
 .../dataflow/sdk/coders/AvroCoderTest.java      |  755 -----
 .../sdk/coders/BigEndianIntegerCoderTest.java   |   91 -
 .../sdk/coders/BigEndianLongCoderTest.java      |   95 -
 .../dataflow/sdk/coders/ByteArrayCoderTest.java |  145 -
 .../dataflow/sdk/coders/ByteCoderTest.java      |   92 -
 .../sdk/coders/ByteStringCoderTest.java         |  122 -
 .../dataflow/sdk/coders/CoderFactoriesTest.java |  101 -
 .../dataflow/sdk/coders/CoderProvidersTest.java |   72 -
 .../dataflow/sdk/coders/CoderRegistryTest.java  |  522 ---
 .../cloud/dataflow/sdk/coders/CoderTest.java    |   80 -
 .../sdk/coders/CollectionCoderTest.java         |   94 -
 .../dataflow/sdk/coders/CustomCoderTest.java    |  136 -
 .../dataflow/sdk/coders/DefaultCoderTest.java   |  129 -
 .../dataflow/sdk/coders/DelegateCoderTest.java  |  142 -
 .../dataflow/sdk/coders/DoubleCoderTest.java    |   97 -
 .../dataflow/sdk/coders/DurationCoderTest.java  |   87 -
 .../dataflow/sdk/coders/EntityCoderTest.java    |  109 -
 .../dataflow/sdk/coders/InstantCoderTest.java   |  117 -
 .../dataflow/sdk/coders/IterableCoderTest.java  |  110 -
 .../dataflow/sdk/coders/JAXBCoderTest.java      |  156 -
 .../cloud/dataflow/sdk/coders/KvCoderTest.java  |  119 -
 .../dataflow/sdk/coders/ListCoderTest.java      |  135 -
 .../cloud/dataflow/sdk/coders/MapCoderTest.java |  107 -
 .../dataflow/sdk/coders/NullableCoderTest.java  |  140 -
 .../sdk/coders/PrintBase64Encodings.java        |   82 -
 .../dataflow/sdk/coders/Proto2CoderTest.java    |  146 -
 .../sdk/coders/SerializableCoderTest.java       |  223 --
 .../cloud/dataflow/sdk/coders/SetCoderTest.java |   87 -
 .../dataflow/sdk/coders/StandardCoderTest.java  |  177 --
 .../sdk/coders/StringDelegateCoderTest.java     |   73 -
 .../sdk/coders/StringUtf8CoderTest.java         |   81 -
 .../sdk/coders/StructuralByteArrayTest.java     |   41 -
 .../sdk/coders/TableRowJsonCoderTest.java       |   87 -
 .../sdk/coders/TextualIntegerCoderTest.java     |   91 -
 .../dataflow/sdk/coders/VarIntCoderTest.java    |   92 -
 .../dataflow/sdk/coders/VarLongCoderTest.java   |   95 -
 .../sdk/coders/protobuf/ProtoCoderTest.java     |  183 --
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |  196 --
 .../sdk/io/AvroIOGeneratedClassTest.java        |  375 ---
 .../cloud/dataflow/sdk/io/AvroIOTest.java       |  259 --
 .../cloud/dataflow/sdk/io/AvroSourceTest.java   |  693 ----
 .../cloud/dataflow/sdk/io/BigQueryIOTest.java   |  606 ----
 .../io/BoundedReadFromUnboundedSourceTest.java  |  133 -
 .../dataflow/sdk/io/CompressedSourceTest.java   |  477 ---
 .../dataflow/sdk/io/CountingInputTest.java      |  152 -
 .../dataflow/sdk/io/CountingSourceTest.java     |  290 --
 .../cloud/dataflow/sdk/io/DatastoreIOTest.java  |  588 ----
 .../dataflow/sdk/io/FileBasedSinkTest.java      |  513 ---
 .../dataflow/sdk/io/FileBasedSourceTest.java    |  917 ------
 .../dataflow/sdk/io/OffsetBasedSourceTest.java  |  281 --
 .../cloud/dataflow/sdk/io/PubsubIOTest.java     |  234 --
 .../google/cloud/dataflow/sdk/io/ReadTest.java  |  145 -
 .../cloud/dataflow/sdk/io/TextIOTest.java       |  517 ---
 .../google/cloud/dataflow/sdk/io/WriteTest.java |  344 --
 .../cloud/dataflow/sdk/io/XmlSinkTest.java      |  236 --
 .../cloud/dataflow/sdk/io/XmlSourceTest.java    |  825 -----
 .../sdk/io/bigtable/BigtableIOTest.java         |  689 ----
 .../range/ByteKeyRangeEstimateFractionTest.java |   70 -
 .../range/ByteKeyRangeInterpolateKeyTest.java   |   74 -
 .../dataflow/sdk/io/range/ByteKeyRangeTest.java |  397 ---
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |  120 -
 .../dataflow/sdk/io/range/ByteKeyTest.java      |  179 --
 .../sdk/io/range/OffsetRangeTrackerTest.java    |  187 --
 .../com/google/cloud/dataflow/sdk/io/user.avsc  |   10 -
 .../dataflow/sdk/options/GcpOptionsTest.java    |  125 -
 .../sdk/options/GoogleApiDebugOptionsTest.java  |  147 -
 .../sdk/options/PipelineOptionsFactoryTest.java | 1150 -------
 .../sdk/options/PipelineOptionsTest.java        |  127 -
 .../options/PipelineOptionsValidatorTest.java   |  311 --
 .../sdk/options/ProxyInvocationHandlerTest.java |  692 ----
 .../AggregatorPipelineExtractorTest.java        |  229 --
 .../runners/DirectPipelineRegistrarTest.java    |   70 -
 .../sdk/runners/DirectPipelineRunnerTest.java   |  211 --
 .../sdk/runners/DirectPipelineTest.java         |   35 -
 .../sdk/runners/PipelineRunnerTest.java         |   81 -
 .../dataflow/sdk/runners/TransformTreeTest.java |  195 --
 .../runners/dataflow/TestCountingSource.java    |  247 --
 .../AvroIOShardedWriteFactoryTest.java          |  112 -
 .../BoundedReadEvaluatorFactoryTest.java        |  290 --
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 --
 .../EncodabilityEnforcementFactoryTest.java     |  262 --
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 -
 .../inprocess/ForwardingPTransformTest.java     |  102 -
 .../GroupByKeyEvaluatorFactoryTest.java         |  182 --
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 --
 .../ImmutabilityEnforcementFactoryTest.java     |  127 -
 .../inprocess/InMemoryWatermarkManagerTest.java | 1124 -------
 .../inprocess/InProcessBundleFactoryTest.java   |  197 --
 .../runners/inprocess/InProcessBundleTest.java  |  145 -
 .../runners/inprocess/InProcessCreateTest.java  |  235 --
 .../InProcessEvaluationContextTest.java         |  535 ----
 .../InProcessPipelineRegistrarTest.java         |   73 -
 .../inprocess/InProcessPipelineRunnerTest.java  |   79 -
 .../InProcessSideInputContainerTest.java        |  372 ---
 .../inprocess/InProcessTimerInternalsTest.java  |  133 -
 .../KeyedPValueTrackingVisitorTest.java         |  191 --
 .../sdk/runners/inprocess/MockClock.java        |   62 -
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 ---
 .../ParDoSingleEvaluatorFactoryTest.java        |  325 --
 .../TextIOShardedWriteFactoryTest.java          |  112 -
 .../TransformExecutorServicesTest.java          |  136 -
 .../inprocess/TransformExecutorTest.java        |  537 ----
 .../UnboundedReadEvaluatorFactoryTest.java      |  333 --
 .../inprocess/ViewEvaluatorFactoryTest.java     |  101 -
 .../WatermarkCallbackExecutorTest.java          |  128 -
 .../inprocess/WindowEvaluatorFactoryTest.java   |  221 --
 .../sdk/testing/CoderPropertiesTest.java        |  215 --
 .../sdk/testing/DataflowJUnitTestRunner.java    |  129 -
 .../dataflow/sdk/testing/ExpectedLogs.java      |  307 --
 .../dataflow/sdk/testing/ExpectedLogsTest.java  |  154 -
 .../sdk/testing/FastNanoClockAndSleeper.java    |   48 -
 .../testing/FastNanoClockAndSleeperTest.java    |   48 -
 .../cloud/dataflow/sdk/testing/PAssertTest.java |  309 --
 .../sdk/testing/PCollectionViewTesting.java     |  296 --
 .../sdk/testing/ResetDateTimeProvider.java      |   42 -
 .../sdk/testing/ResetDateTimeProviderTest.java  |   56 -
 .../sdk/testing/RestoreSystemProperties.java    |   50 -
 .../testing/RestoreSystemPropertiesTest.java    |   51 -
 .../sdk/testing/SerializableMatchersTest.java   |  166 -
 .../sdk/testing/SystemNanoTimeSleeper.java      |   69 -
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |   54 -
 .../dataflow/sdk/testing/TestPipelineTest.java  |  103 -
 .../transforms/ApproximateQuantilesTest.java    |  298 --
 .../sdk/transforms/ApproximateUniqueTest.java   |  292 --
 .../dataflow/sdk/transforms/CombineFnsTest.java |  415 ---
 .../dataflow/sdk/transforms/CombineTest.java    | 1138 -------
 .../dataflow/sdk/transforms/CountTest.java      |  122 -
 .../dataflow/sdk/transforms/CreateTest.java     |  241 --
 .../sdk/transforms/DoFnContextTest.java         |   69 -
 .../DoFnDelegatingAggregatorTest.java           |  144 -
 .../sdk/transforms/DoFnReflectorTest.java       |  494 ---
 .../cloud/dataflow/sdk/transforms/DoFnTest.java |  202 --
 .../dataflow/sdk/transforms/DoFnTesterTest.java |  255 --
 .../sdk/transforms/DoFnWithContextTest.java     |  221 --
 .../dataflow/sdk/transforms/FilterTest.java     |  161 -
 .../sdk/transforms/FlatMapElementsTest.java     |  125 -
 .../dataflow/sdk/transforms/FlattenTest.java    |  370 ---
 .../dataflow/sdk/transforms/GroupByKeyTest.java |  377 ---
 .../IntraBundleParallelizationTest.java         |  251 --
 .../cloud/dataflow/sdk/transforms/KeysTest.java |   84 -
 .../dataflow/sdk/transforms/KvSwapTest.java     |   92 -
 .../sdk/transforms/MapElementsTest.java         |  135 -
 .../cloud/dataflow/sdk/transforms/MaxTest.java  |   67 -
 .../cloud/dataflow/sdk/transforms/MeanTest.java |   73 -
 .../cloud/dataflow/sdk/transforms/MinTest.java  |   67 -
 .../cloud/dataflow/sdk/transforms/NoOpDoFn.java |  144 -
 .../dataflow/sdk/transforms/PTransformTest.java |   42 -
 .../dataflow/sdk/transforms/ParDoTest.java      | 1542 ---------
 .../dataflow/sdk/transforms/PartitionTest.java  |  141 -
 .../sdk/transforms/RemoveDuplicatesTest.java    |  132 -
 .../dataflow/sdk/transforms/SampleTest.java     |  261 --
 .../sdk/transforms/SimpleStatsFnsTest.java      |  130 -
 .../cloud/dataflow/sdk/transforms/SumTest.java  |   68 -
 .../cloud/dataflow/sdk/transforms/TopTest.java  |  260 --
 .../dataflow/sdk/transforms/ValuesTest.java     |   94 -
 .../cloud/dataflow/sdk/transforms/ViewTest.java | 1423 ---------
 .../dataflow/sdk/transforms/WithKeysTest.java   |  128 -
 .../sdk/transforms/WithTimestampsTest.java      |  211 --
 .../transforms/display/DisplayDataMatchers.java |  321 --
 .../display/DisplayDataMatchersTest.java        |  157 -
 .../sdk/transforms/display/DisplayDataTest.java |  955 ------
 .../transforms/join/CoGbkResultCoderTest.java   |   86 -
 .../sdk/transforms/join/CoGbkResultTest.java    |  125 -
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  508 ---
 .../sdk/transforms/join/UnionCoderTest.java     |   49 -
 .../sdk/transforms/windowing/AfterAllTest.java  |  152 -
 .../sdk/transforms/windowing/AfterEachTest.java |  123 -
 .../transforms/windowing/AfterFirstTest.java    |  176 -
 .../sdk/transforms/windowing/AfterPaneTest.java |  127 -
 .../windowing/AfterProcessingTimeTest.java      |  158 -
 .../AfterSynchronizedProcessingTimeTest.java    |  122 -
 .../windowing/AfterWatermarkTest.java           |  339 --
 .../windowing/CalendarWindowsTest.java          |  261 --
 .../windowing/DefaultTriggerTest.java           |  177 --
 .../transforms/windowing/FixedWindowsTest.java  |  125 -
 .../windowing/IntervalWindowTest.java           |   95 -
 .../windowing/OrFinallyTriggerTest.java         |  210 --
 .../sdk/transforms/windowing/PaneInfoTest.java  |   77 -
 .../transforms/windowing/RepeatedlyTest.java    |  212 --
 .../sdk/transforms/windowing/SessionsTest.java  |  157 -
 .../windowing/SlidingWindowsTest.java           |  194 --
 .../sdk/transforms/windowing/TriggerTest.java   |  119 -
 .../sdk/transforms/windowing/WindowTest.java    |  227 --
 .../sdk/transforms/windowing/WindowingTest.java |  245 --
 .../cloud/dataflow/sdk/util/ApiSurfaceTest.java |  188 --
 ...mptAndTimeBoundedExponentialBackOffTest.java |  213 --
 .../AttemptBoundedExponentialBackOffTest.java   |   86 -
 .../cloud/dataflow/sdk/util/AvroUtilsTest.java  |  226 --
 .../sdk/util/BatchTimerInternalsTest.java       |  118 -
 .../sdk/util/BigQueryServicesImplTest.java      |  262 --
 .../sdk/util/BigQueryTableInserterTest.java     |  304 --
 .../sdk/util/BigQueryTableRowIteratorTest.java  |  256 --
 .../dataflow/sdk/util/BigQueryUtilTest.java     |  480 ---
 ...BufferedElementCountingOutputStreamTest.java |  208 --
 .../cloud/dataflow/sdk/util/CoderUtilsTest.java |  230 --
 .../dataflow/sdk/util/CombineFnUtilTest.java    |  104 -
 .../sdk/util/CounterAggregatorTest.java         |  254 --
 .../sdk/util/ExecutableTriggerTest.java         |  129 -
 .../util/ExposedByteArrayInputStreamTest.java   |   80 -
 .../util/ExposedByteArrayOutputStreamTest.java  |  247 --
 .../sdk/util/FileIOChannelFactoryTest.java      |  227 --
 .../sdk/util/FinishedTriggersBitSetTest.java    |   55 -
 .../sdk/util/FinishedTriggersProperties.java    |  110 -
 .../sdk/util/FinishedTriggersSetTest.java       |   61 -
 .../sdk/util/GcsIOChannelFactoryTest.java       |   44 -
 .../cloud/dataflow/sdk/util/GcsUtilTest.java    |  492 ---
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  719 -----
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  112 -
 .../dataflow/sdk/util/IOChannelUtilsTest.java   |   95 -
 .../dataflow/sdk/util/InstanceBuilderTest.java  |  116 -
 .../IntervalBoundedExponentialBackOffTest.java  |  100 -
 .../sdk/util/KeyedWorkItemCoderTest.java        |   63 -
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 -
 .../sdk/util/MergingActiveWindowSetTest.java    |  174 -
 .../sdk/util/MutationDetectorsTest.java         |  149 -
 .../cloud/dataflow/sdk/util/PTupleTest.java     |   41 -
 .../dataflow/sdk/util/RandomAccessDataTest.java |  207 --
 .../dataflow/sdk/util/ReduceFnRunnerTest.java   | 1205 -------
 .../cloud/dataflow/sdk/util/ReduceFnTester.java |  779 -----
 .../cloud/dataflow/sdk/util/ReshuffleTest.java  |  209 --
 .../dataflow/sdk/util/ReshuffleTriggerTest.java |   60 -
 .../util/RetryHttpRequestInitializerTest.java   |  297 --
 .../sdk/util/SerializableUtilsTest.java         |  166 -
 .../cloud/dataflow/sdk/util/SerializerTest.java |  163 -
 .../dataflow/sdk/util/SimpleDoFnRunnerTest.java |   87 -
 .../dataflow/sdk/util/StreamUtilsTest.java      |   73 -
 .../dataflow/sdk/util/StringUtilsTest.java      |  146 -
 .../cloud/dataflow/sdk/util/StructsTest.java    |  207 --
 .../cloud/dataflow/sdk/util/TimeUtilTest.java   |   74 -
 .../dataflow/sdk/util/TimerInternalsTest.java   |   54 -
 .../cloud/dataflow/sdk/util/TriggerTester.java  |  584 ----
 .../sdk/util/UnownedInputStreamTest.java        |   77 -
 .../sdk/util/UnownedOutputStreamTest.java       |   58 -
 .../util/UploadIdResponseInterceptorTest.java   |  101 -
 .../sdk/util/UserCodeExceptionTest.java         |  176 -
 .../cloud/dataflow/sdk/util/VarIntTest.java     |  278 --
 .../dataflow/sdk/util/WindowedValueTest.java    |   58 -
 .../cloud/dataflow/sdk/util/ZipFilesTest.java   |  312 --
 .../sdk/util/common/CounterSetTest.java         |  226 --
 .../dataflow/sdk/util/common/CounterTest.java   |  590 ----
 .../sdk/util/common/CounterTestUtils.java       |   57 -
 .../sdk/util/common/ReflectHelpersTest.java     |  128 -
 .../dataflow/sdk/util/gcsfs/GcsPathTest.java    |  334 --
 .../CopyOnAccessInMemoryStateInternalsTest.java |  555 ----
 .../util/state/InMemoryStateInternalsTest.java  |  350 --
 .../sdk/util/state/StateNamespacesTest.java     |  131 -
 .../dataflow/sdk/util/state/StateTagTest.java   |  175 -
 .../cloud/dataflow/sdk/values/KVTest.java       |  113 -
 .../sdk/values/PCollectionListTest.java         |   48 -
 .../sdk/values/PCollectionTupleTest.java        |   94 -
 .../cloud/dataflow/sdk/values/PDoneTest.java    |  103 -
 .../cloud/dataflow/sdk/values/TupleTagTest.java |   88 -
 .../dataflow/sdk/values/TypeDescriptorTest.java |  194 --
 .../dataflow/sdk/values/TypedPValueTest.java    |  165 -
 .../org/apache/beam/sdk/DataflowMatchers.java   |   66 +
 .../java/org/apache/beam/sdk/PipelineTest.java  |  297 ++
 .../java/org/apache/beam/sdk/TestUtils.java     |  214 ++
 .../org/apache/beam/sdk/WindowMatchers.java     |  138 +
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  755 +++++
 .../sdk/coders/BigEndianIntegerCoderTest.java   |   91 +
 .../beam/sdk/coders/BigEndianLongCoderTest.java |   95 +
 .../beam/sdk/coders/ByteArrayCoderTest.java     |  145 +
 .../apache/beam/sdk/coders/ByteCoderTest.java   |   92 +
 .../beam/sdk/coders/ByteStringCoderTest.java    |  122 +
 .../beam/sdk/coders/CoderFactoriesTest.java     |  101 +
 .../beam/sdk/coders/CoderProvidersTest.java     |   72 +
 .../beam/sdk/coders/CoderRegistryTest.java      |  522 +++
 .../org/apache/beam/sdk/coders/CoderTest.java   |   79 +
 .../beam/sdk/coders/CollectionCoderTest.java    |   94 +
 .../apache/beam/sdk/coders/CustomCoderTest.java |  136 +
 .../beam/sdk/coders/DefaultCoderTest.java       |  129 +
 .../beam/sdk/coders/DelegateCoderTest.java      |  142 +
 .../apache/beam/sdk/coders/DoubleCoderTest.java |   97 +
 .../beam/sdk/coders/DurationCoderTest.java      |   87 +
 .../apache/beam/sdk/coders/EntityCoderTest.java |  109 +
 .../beam/sdk/coders/InstantCoderTest.java       |  117 +
 .../beam/sdk/coders/IterableCoderTest.java      |  110 +
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |  156 +
 .../org/apache/beam/sdk/coders/KvCoderTest.java |  119 +
 .../apache/beam/sdk/coders/ListCoderTest.java   |  135 +
 .../apache/beam/sdk/coders/MapCoderTest.java    |  107 +
 .../beam/sdk/coders/NullableCoderTest.java      |  140 +
 .../beam/sdk/coders/PrintBase64Encodings.java   |   82 +
 .../apache/beam/sdk/coders/Proto2CoderTest.java |  146 +
 .../beam/sdk/coders/SerializableCoderTest.java  |  223 ++
 .../apache/beam/sdk/coders/SetCoderTest.java    |   87 +
 .../beam/sdk/coders/StandardCoderTest.java      |  177 ++
 .../sdk/coders/StringDelegateCoderTest.java     |   73 +
 .../beam/sdk/coders/StringUtf8CoderTest.java    |   81 +
 .../sdk/coders/StructuralByteArrayTest.java     |   41 +
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |   87 +
 .../sdk/coders/TextualIntegerCoderTest.java     |   91 +
 .../apache/beam/sdk/coders/VarIntCoderTest.java |   91 +
 .../beam/sdk/coders/VarLongCoderTest.java       |   95 +
 .../sdk/coders/protobuf/ProtoCoderTest.java     |  183 ++
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |  196 ++
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  375 +++
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  259 ++
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  693 ++++
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  606 ++++
 .../io/BoundedReadFromUnboundedSourceTest.java  |  133 +
 .../beam/sdk/io/CompressedSourceTest.java       |  477 +++
 .../apache/beam/sdk/io/CountingInputTest.java   |  152 +
 .../apache/beam/sdk/io/CountingSourceTest.java  |  290 ++
 .../org/apache/beam/sdk/io/DatastoreIOTest.java |  588 ++++
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  513 +++
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  917 ++++++
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  281 ++
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  234 ++
 .../java/org/apache/beam/sdk/io/ReadTest.java   |  145 +
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  517 +++
 .../java/org/apache/beam/sdk/io/WriteTest.java  |  344 ++
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  236 ++
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  825 +++++
 .../beam/sdk/io/bigtable/BigtableIOTest.java    |  689 ++++
 .../range/ByteKeyRangeEstimateFractionTest.java |   70 +
 .../range/ByteKeyRangeInterpolateKeyTest.java   |   74 +
 .../beam/sdk/io/range/ByteKeyRangeTest.java     |  397 +++
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |  120 +
 .../apache/beam/sdk/io/range/ByteKeyTest.java   |  179 ++
 .../sdk/io/range/OffsetRangeTrackerTest.java    |  187 ++
 .../test/java/org/apache/beam/sdk/io/user.avsc  |   10 +
 .../apache/beam/sdk/options/GcpOptionsTest.java |  124 +
 .../sdk/options/GoogleApiDebugOptionsTest.java  |  147 +
 .../sdk/options/PipelineOptionsFactoryTest.java | 1150 +++++++
 .../beam/sdk/options/PipelineOptionsTest.java   |  126 +
 .../options/PipelineOptionsValidatorTest.java   |  311 ++
 .../sdk/options/ProxyInvocationHandlerTest.java |  692 ++++
 .../AggregatorPipelineExtractorTest.java        |  228 ++
 .../runners/DirectPipelineRegistrarTest.java    |   70 +
 .../sdk/runners/DirectPipelineRunnerTest.java   |  211 ++
 .../beam/sdk/runners/DirectPipelineTest.java    |   35 +
 .../beam/sdk/runners/PipelineRunnerTest.java    |   81 +
 .../beam/sdk/runners/TransformTreeTest.java     |  195 ++
 .../runners/dataflow/TestCountingSource.java    |  247 ++
 .../AvroIOShardedWriteFactoryTest.java          |  112 +
 .../BoundedReadEvaluatorFactoryTest.java        |  289 ++
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ++
 .../EncodabilityEnforcementFactoryTest.java     |  262 ++
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 +
 .../inprocess/ForwardingPTransformTest.java     |  102 +
 .../GroupByKeyEvaluatorFactoryTest.java         |  182 ++
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 ++
 .../ImmutabilityEnforcementFactoryTest.java     |  126 +
 .../inprocess/InMemoryWatermarkManagerTest.java | 1124 +++++++
 .../inprocess/InProcessBundleFactoryTest.java   |  197 ++
 .../runners/inprocess/InProcessBundleTest.java  |  144 +
 .../runners/inprocess/InProcessCreateTest.java  |  235 ++
 .../InProcessEvaluationContextTest.java         |  535 ++++
 .../InProcessPipelineRegistrarTest.java         |   73 +
 .../inprocess/InProcessPipelineRunnerTest.java  |   78 +
 .../InProcessSideInputContainerTest.java        |  372 +++
 .../inprocess/InProcessTimerInternalsTest.java  |  133 +
 .../KeyedPValueTrackingVisitorTest.java         |  191 ++
 .../beam/sdk/runners/inprocess/MockClock.java   |   62 +
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 +++
 .../ParDoSingleEvaluatorFactoryTest.java        |  324 ++
 .../TextIOShardedWriteFactoryTest.java          |  112 +
 .../TransformExecutorServicesTest.java          |  136 +
 .../inprocess/TransformExecutorTest.java        |  537 ++++
 .../UnboundedReadEvaluatorFactoryTest.java      |  333 ++
 .../inprocess/ViewEvaluatorFactoryTest.java     |  100 +
 .../WatermarkCallbackExecutorTest.java          |  128 +
 .../inprocess/WindowEvaluatorFactoryTest.java   |  221 ++
 .../beam/sdk/testing/CoderPropertiesTest.java   |  215 ++
 .../sdk/testing/DataflowJUnitTestRunner.java    |  129 +
 .../apache/beam/sdk/testing/ExpectedLogs.java   |  307 ++
 .../beam/sdk/testing/ExpectedLogsTest.java      |  154 +
 .../sdk/testing/FastNanoClockAndSleeper.java    |   48 +
 .../testing/FastNanoClockAndSleeperTest.java    |   48 +
 .../apache/beam/sdk/testing/PAssertTest.java    |  309 ++
 .../sdk/testing/PCollectionViewTesting.java     |  296 ++
 .../beam/sdk/testing/ResetDateTimeProvider.java |   42 +
 .../sdk/testing/ResetDateTimeProviderTest.java  |   56 +
 .../sdk/testing/RestoreSystemProperties.java    |   50 +
 .../testing/RestoreSystemPropertiesTest.java    |   51 +
 .../sdk/testing/SerializableMatchersTest.java   |  166 +
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |   69 +
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |   54 +
 .../beam/sdk/testing/TestPipelineTest.java      |  103 +
 .../transforms/ApproximateQuantilesTest.java    |  298 ++
 .../sdk/transforms/ApproximateUniqueTest.java   |  292 ++
 .../beam/sdk/transforms/CombineFnsTest.java     |  415 +++
 .../apache/beam/sdk/transforms/CombineTest.java | 1138 +++++++
 .../apache/beam/sdk/transforms/CountTest.java   |  122 +
 .../apache/beam/sdk/transforms/CreateTest.java  |  241 ++
 .../beam/sdk/transforms/DoFnContextTest.java    |   69 +
 .../DoFnDelegatingAggregatorTest.java           |  144 +
 .../beam/sdk/transforms/DoFnReflectorTest.java  |  494 +++
 .../apache/beam/sdk/transforms/DoFnTest.java    |  202 ++
 .../beam/sdk/transforms/DoFnTesterTest.java     |  254 ++
 .../sdk/transforms/DoFnWithContextTest.java     |  221 ++
 .../apache/beam/sdk/transforms/FilterTest.java  |  161 +
 .../sdk/transforms/FlatMapElementsTest.java     |  125 +
 .../apache/beam/sdk/transforms/FlattenTest.java |  370 +++
 .../beam/sdk/transforms/GroupByKeyTest.java     |  377 +++
 .../IntraBundleParallelizationTest.java         |  251 ++
 .../apache/beam/sdk/transforms/KeysTest.java    |   84 +
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   92 +
 .../beam/sdk/transforms/MapElementsTest.java    |  135 +
 .../org/apache/beam/sdk/transforms/MaxTest.java |   67 +
 .../apache/beam/sdk/transforms/MeanTest.java    |   73 +
 .../org/apache/beam/sdk/transforms/MinTest.java |   67 +
 .../apache/beam/sdk/transforms/NoOpDoFn.java    |  144 +
 .../beam/sdk/transforms/PTransformTest.java     |   42 +
 .../apache/beam/sdk/transforms/ParDoTest.java   | 1542 +++++++++
 .../beam/sdk/transforms/PartitionTest.java      |  141 +
 .../sdk/transforms/RemoveDuplicatesTest.java    |  132 +
 .../apache/beam/sdk/transforms/SampleTest.java  |  261 ++
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |  130 +
 .../org/apache/beam/sdk/transforms/SumTest.java |   68 +
 .../org/apache/beam/sdk/transforms/TopTest.java |  260 ++
 .../apache/beam/sdk/transforms/ValuesTest.java  |   94 +
 .../apache/beam/sdk/transforms/ViewTest.java    | 1423 +++++++++
 .../beam/sdk/transforms/WithKeysTest.java       |  128 +
 .../beam/sdk/transforms/WithTimestampsTest.java |  211 ++
 .../transforms/display/DisplayDataMatchers.java |  321 ++
 .../display/DisplayDataMatchersTest.java        |  157 +
 .../sdk/transforms/display/DisplayDataTest.java |  955 ++++++
 .../transforms/join/CoGbkResultCoderTest.java   |   86 +
 .../sdk/transforms/join/CoGbkResultTest.java    |  125 +
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  508 +++
 .../sdk/transforms/join/UnionCoderTest.java     |   49 +
 .../sdk/transforms/windowing/AfterAllTest.java  |  152 +
 .../sdk/transforms/windowing/AfterEachTest.java |  123 +
 .../transforms/windowing/AfterFirstTest.java    |  176 +
 .../sdk/transforms/windowing/AfterPaneTest.java |  127 +
 .../windowing/AfterProcessingTimeTest.java      |  158 +
 .../AfterSynchronizedProcessingTimeTest.java    |  122 +
 .../windowing/AfterWatermarkTest.java           |  339 ++
 .../windowing/CalendarWindowsTest.java          |  261 ++
 .../windowing/DefaultTriggerTest.java           |  177 ++
 .../transforms/windowing/FixedWindowsTest.java  |  125 +
 .../windowing/IntervalWindowTest.java           |   95 +
 .../windowing/OrFinallyTriggerTest.java         |  210 ++
 .../sdk/transforms/windowing/PaneInfoTest.java  |   77 +
 .../transforms/windowing/RepeatedlyTest.java    |  212 ++
 .../sdk/transforms/windowing/SessionsTest.java  |  157 +
 .../windowing/SlidingWindowsTest.java           |  194 ++
 .../sdk/transforms/windowing/TriggerTest.java   |  119 +
 .../sdk/transforms/windowing/WindowTest.java    |  227 ++
 .../sdk/transforms/windowing/WindowingTest.java |  245 ++
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  188 ++
 ...mptAndTimeBoundedExponentialBackOffTest.java |  213 ++
 .../AttemptBoundedExponentialBackOffTest.java   |   86 +
 .../org/apache/beam/sdk/util/AvroUtilsTest.java |  226 ++
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  118 +
 .../beam/sdk/util/BigQueryServicesImplTest.java |  261 ++
 .../sdk/util/BigQueryTableInserterTest.java     |  304 ++
 .../sdk/util/BigQueryTableRowIteratorTest.java  |  256 ++
 .../apache/beam/sdk/util/BigQueryUtilTest.java  |  480 +++
 ...BufferedElementCountingOutputStreamTest.java |  207 ++
 .../apache/beam/sdk/util/CoderUtilsTest.java    |  230 ++
 .../apache/beam/sdk/util/CombineFnUtilTest.java |  104 +
 .../beam/sdk/util/CounterAggregatorTest.java    |  254 ++
 .../beam/sdk/util/ExecutableTriggerTest.java    |  129 +
 .../util/ExposedByteArrayInputStreamTest.java   |   80 +
 .../util/ExposedByteArrayOutputStreamTest.java  |  247 ++
 .../beam/sdk/util/FileIOChannelFactoryTest.java |  227 ++
 .../sdk/util/FinishedTriggersBitSetTest.java    |   55 +
 .../sdk/util/FinishedTriggersProperties.java    |  110 +
 .../beam/sdk/util/FinishedTriggersSetTest.java  |   61 +
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |   44 +
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  492 +++
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  719 +++++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  112 +
 .../beam/sdk/util/IOChannelUtilsTest.java       |   95 +
 .../beam/sdk/util/InstanceBuilderTest.java      |  116 +
 .../IntervalBoundedExponentialBackOffTest.java  |  100 +
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |   63 +
 .../util/LateDataDroppingDoFnRunnerTest.java    |  116 +
 .../sdk/util/MergingActiveWindowSetTest.java    |  174 +
 .../beam/sdk/util/MutationDetectorsTest.java    |  148 +
 .../org/apache/beam/sdk/util/PTupleTest.java    |   41 +
 .../beam/sdk/util/RandomAccessDataTest.java     |  206 ++
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1205 +++++++
 .../apache/beam/sdk/util/ReduceFnTester.java    |  779 +++++
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  209 ++
 .../beam/sdk/util/ReshuffleTriggerTest.java     |   60 +
 .../util/RetryHttpRequestInitializerTest.java   |  297 ++
 .../beam/sdk/util/SerializableUtilsTest.java    |  166 +
 .../apache/beam/sdk/util/SerializerTest.java    |  163 +
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   86 +
 .../apache/beam/sdk/util/StreamUtilsTest.java   |   73 +
 .../apache/beam/sdk/util/StringUtilsTest.java   |  146 +
 .../org/apache/beam/sdk/util/StructsTest.java   |  207 ++
 .../org/apache/beam/sdk/util/TimeUtilTest.java  |   74 +
 .../beam/sdk/util/TimerInternalsTest.java       |   53 +
 .../org/apache/beam/sdk/util/TriggerTester.java |  584 ++++
 .../beam/sdk/util/UnownedInputStreamTest.java   |   76 +
 .../beam/sdk/util/UnownedOutputStreamTest.java  |   57 +
 .../util/UploadIdResponseInterceptorTest.java   |  101 +
 .../beam/sdk/util/UserCodeExceptionTest.java    |  175 +
 .../org/apache/beam/sdk/util/VarIntTest.java    |  278 ++
 .../apache/beam/sdk/util/WindowedValueTest.java |   58 +
 .../org/apache/beam/sdk/util/ZipFilesTest.java  |  312 ++
 .../beam/sdk/util/common/CounterSetTest.java    |  226 ++
 .../beam/sdk/util/common/CounterTest.java       |  590 ++++
 .../beam/sdk/util/common/CounterTestUtils.java  |   57 +
 .../sdk/util/common/ReflectHelpersTest.java     |  128 +
 .../apache/beam/sdk/util/gcsfs/GcsPathTest.java |  334 ++
 .../CopyOnAccessInMemoryStateInternalsTest.java |  555 ++++
 .../util/state/InMemoryStateInternalsTest.java  |  350 ++
 .../sdk/util/state/StateNamespacesTest.java     |  131 +
 .../beam/sdk/util/state/StateTagTest.java       |  175 +
 .../java/org/apache/beam/sdk/values/KVTest.java |  113 +
 .../beam/sdk/values/PCollectionListTest.java    |   48 +
 .../beam/sdk/values/PCollectionTupleTest.java   |   94 +
 .../org/apache/beam/sdk/values/PDoneTest.java   |  103 +
 .../apache/beam/sdk/values/TupleTagTest.java    |   88 +
 .../beam/sdk/values/TypeDescriptorTest.java     |  194 ++
 .../apache/beam/sdk/values/TypedPValueTest.java |  164 +
 .../PipelineOptionsFactoryJava8Test.java        |   92 -
 .../PipelineOptionsFactoryJava8Test.java        |   92 +
 .../sdk/transforms/CombineJava8Test.java        |  134 -
 .../sdk/transforms/FilterJava8Test.java         |  119 -
 .../transforms/FlatMapElementsJava8Test.java    |   85 -
 .../sdk/transforms/MapElementsJava8Test.java    |   78 -
 .../sdk/transforms/PartitionJava8Test.java      |   75 -
 .../transforms/RemoveDuplicatesJava8Test.java   |  100 -
 .../sdk/transforms/WithKeysJava8Test.java       |   75 -
 .../sdk/transforms/WithTimestampsJava8Test.java |   67 -
 .../beam/sdk/transforms/CombineJava8Test.java   |  134 +
 .../beam/sdk/transforms/FilterJava8Test.java    |  119 +
 .../transforms/FlatMapElementsJava8Test.java    |   85 +
 .../sdk/transforms/MapElementsJava8Test.java    |   78 +
 .../beam/sdk/transforms/PartitionJava8Test.java |   75 +
 .../transforms/RemoveDuplicatesJava8Test.java   |  100 +
 .../beam/sdk/transforms/WithKeysJava8Test.java  |   75 +
 .../sdk/transforms/WithTimestampsJava8Test.java |   67 +
 1646 files changed, 174737 insertions(+), 174807 deletions(-)
----------------------------------------------------------------------




[23/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
new file mode 100644
index 0000000..d2c08c8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
@@ -0,0 +1,483 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+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 com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpResponse;
+import com.google.api.client.http.HttpStatusCodes;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.LowLevelHttpRequest;
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.Json;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.client.testing.http.HttpTesting;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.cloud.dataflow.sdk.options.GcsOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
+import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
+import com.google.cloud.dataflow.sdk.util.PackageUtil.PackageAttributes;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.io.LineReader;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+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 org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.Pipe;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/** Tests for PackageUtil. */
+@RunWith(JUnit4.class)
+public class PackageUtilTest {
+  @Rule public ExpectedLogs logged = ExpectedLogs.none(PackageUtil.class);
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Rule
+  public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper();
+
+  @Mock
+  GcsUtil mockGcsUtil;
+
+  // 128 bits, base64 encoded is 171 bits, rounds to 22 bytes
+  private static final String HASH_PATTERN = "[a-zA-Z0-9+-]{22}";
+
+  // Hamcrest matcher to assert a string matches a pattern
+  private static class RegexMatcher extends BaseMatcher<String> {
+    private final Pattern pattern;
+
+    public RegexMatcher(String regex) {
+      this.pattern = Pattern.compile(regex);
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      if (!(o instanceof String)) {
+        return false;
+      }
+      return pattern.matcher((String) o).matches();
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText(String.format("matches regular expression %s", pattern));
+    }
+
+    public static RegexMatcher matches(String regex) {
+      return new RegexMatcher(regex);
+    }
+  }
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+
+    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
+    pipelineOptions.setGcsUtil(mockGcsUtil);
+
+    IOChannelUtils.registerStandardIOFactories(pipelineOptions);
+  }
+
+  private File makeFileWithContents(String name, String contents) throws Exception {
+    File tmpFile = tmpFolder.newFile(name);
+    Files.write(contents, tmpFile, StandardCharsets.UTF_8);
+    tmpFile.setLastModified(0);  // required for determinism with directories
+    return tmpFile;
+  }
+
+  static final String STAGING_PATH = GcsPath.fromComponents("somebucket", "base/path").toString();
+  private static PackageAttributes makePackageAttributes(File file, String overridePackageName) {
+    return PackageUtil.createPackageAttributes(file, STAGING_PATH, overridePackageName);
+  }
+
+  @Test
+  public void testFileWithExtensionPackageNamingAndSize() throws Exception {
+    String contents = "This is a test!";
+    File tmpFile = makeFileWithContents("file.txt", contents);
+    PackageAttributes attr = makePackageAttributes(tmpFile, null);
+    DataflowPackage target = attr.getDataflowPackage();
+
+    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
+    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+    assertThat(attr.getSize(), equalTo((long) contents.length()));
+  }
+
+  @Test
+  public void testPackageNamingWithFileNoExtension() throws Exception {
+    File tmpFile = makeFileWithContents("file", "This is a test!");
+    DataflowPackage target = makePackageAttributes(tmpFile, null).getDataflowPackage();
+
+    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN));
+    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+  }
+
+  @Test
+  public void testPackageNamingWithDirectory() throws Exception {
+    File tmpDirectory = tmpFolder.newFolder("folder");
+    DataflowPackage target = makePackageAttributes(tmpDirectory, null).getDataflowPackage();
+
+    assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
+    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+  }
+
+  @Test
+  public void testPackageNamingWithFilesHavingSameContentsAndSameNames() throws Exception {
+    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+    makeFileWithContents("folder1/folderA/sameName", "This is a test!");
+    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+    makeFileWithContents("folder2/folderA/sameName", "This is a test!");
+    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+    assertEquals(target1.getName(), target2.getName());
+    assertEquals(target1.getLocation(), target2.getLocation());
+  }
+
+  @Test
+  public void testPackageNamingWithFilesHavingSameContentsButDifferentNames() throws Exception {
+    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+    makeFileWithContents("folder1/folderA/uniqueName1", "This is a test!");
+    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+    makeFileWithContents("folder2/folderA/uniqueName2", "This is a test!");
+    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+    assertNotEquals(target1.getName(), target2.getName());
+    assertNotEquals(target1.getLocation(), target2.getLocation());
+  }
+
+  @Test
+  public void testPackageNamingWithDirectoriesHavingSameContentsButDifferentNames()
+      throws Exception {
+    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+    tmpFolder.newFolder("folder1", "folderA", "uniqueName1");
+    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+    tmpFolder.newFolder("folder2", "folderA", "uniqueName2");
+    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+    assertNotEquals(target1.getName(), target2.getName());
+    assertNotEquals(target1.getLocation(), target2.getLocation());
+  }
+
+  @Test
+  public void testPackageUploadWithLargeClasspathLogsWarning() throws Exception {
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    // all files will be present and cached so no upload needed.
+    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
+
+    List<String> classpathElements = Lists.newLinkedList();
+    for (int i = 0; i < 1005; ++i) {
+      String eltName = "element" + i;
+      classpathElements.add(eltName + '=' + tmpFile.getAbsolutePath());
+    }
+
+    PackageUtil.stageClasspathElements(classpathElements, STAGING_PATH);
+
+    logged.verifyWarn("Your classpath contains 1005 elements, which Google Cloud Dataflow");
+  }
+
+  @Test
+  public void testPackageUploadWithFileSucceeds() throws Exception {
+    Pipe pipe = Pipe.open();
+    String contents = "This is a test!";
+    File tmpFile = makeFileWithContents("file.txt", contents);
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+        ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
+    DataflowPackage target = Iterables.getOnlyElement(targets);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+    verifyNoMoreInteractions(mockGcsUtil);
+
+    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
+    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+    assertThat(new LineReader(Channels.newReader(pipe.source(), "UTF-8")).readLine(),
+        equalTo(contents));
+  }
+
+  @Test
+  public void testPackageUploadWithDirectorySucceeds() throws Exception {
+    Pipe pipe = Pipe.open();
+    File tmpDirectory = tmpFolder.newFolder("folder");
+    tmpFolder.newFolder("folder", "empty_directory");
+    tmpFolder.newFolder("folder", "directory");
+    makeFileWithContents("folder/file.txt", "This is a test!");
+    makeFileWithContents("folder/directory/file.txt", "This is also a test!");
+
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+    PackageUtil.stageClasspathElements(
+        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+    verifyNoMoreInteractions(mockGcsUtil);
+
+    ZipInputStream inputStream = new ZipInputStream(Channels.newInputStream(pipe.source()));
+    List<String> zipEntryNames = new ArrayList<>();
+    for (ZipEntry entry = inputStream.getNextEntry(); entry != null;
+        entry = inputStream.getNextEntry()) {
+      zipEntryNames.add(entry.getName());
+    }
+
+    assertThat(zipEntryNames,
+        containsInAnyOrder("directory/file.txt", "empty_directory/", "file.txt"));
+  }
+
+  @Test
+  public void testPackageUploadWithEmptyDirectorySucceeds() throws Exception {
+    Pipe pipe = Pipe.open();
+    File tmpDirectory = tmpFolder.newFolder("folder");
+
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+    DataflowPackage target = Iterables.getOnlyElement(targets);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+    verifyNoMoreInteractions(mockGcsUtil);
+
+    assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
+    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+    assertNull(new ZipInputStream(Channels.newInputStream(pipe.source())).getNextEntry());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception {
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+        .thenThrow(new IOException("Fake Exception: Upload error"));
+
+    try {
+      PackageUtil.stageClasspathElements(
+          ImmutableList.of(tmpFile.getAbsolutePath()),
+          STAGING_PATH, fastNanoClockAndSleeper);
+    } finally {
+      verify(mockGcsUtil).fileSize(any(GcsPath.class));
+      verify(mockGcsUtil, times(5)).create(any(GcsPath.class), anyString());
+      verifyNoMoreInteractions(mockGcsUtil);
+    }
+  }
+
+  @Test
+  public void testPackageUploadFailsWithPermissionsErrorGivesDetailedMessage() throws Exception {
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+        .thenThrow(new IOException("Failed to write to GCS path " + STAGING_PATH,
+            googleJsonResponseException(
+                HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Permission denied", "Test message")));
+
+    try {
+      PackageUtil.stageClasspathElements(
+          ImmutableList.of(tmpFile.getAbsolutePath()),
+          STAGING_PATH, fastNanoClockAndSleeper);
+      fail("Expected RuntimeException");
+    } catch (RuntimeException e) {
+      assertTrue("Expected IOException containing detailed message.",
+          e.getCause() instanceof IOException);
+      assertThat(e.getCause().getMessage(),
+          Matchers.allOf(
+              Matchers.containsString("Uploaded failed due to permissions error"),
+              Matchers.containsString(
+                  "Stale credentials can be resolved by executing 'gcloud auth login'")));
+    } finally {
+      verify(mockGcsUtil).fileSize(any(GcsPath.class));
+      verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+      verifyNoMoreInteractions(mockGcsUtil);
+    }
+  }
+
+  @Test
+  public void testPackageUploadEventuallySucceeds() throws Exception {
+    Pipe pipe = Pipe.open();
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+        .thenThrow(new IOException("Fake Exception: 410 Gone")) // First attempt fails
+        .thenReturn(pipe.sink());                               // second attempt succeeds
+
+    try {
+      PackageUtil.stageClasspathElements(
+                                              ImmutableList.of(tmpFile.getAbsolutePath()),
+                                              STAGING_PATH,
+                                              fastNanoClockAndSleeper);
+    } finally {
+      verify(mockGcsUtil).fileSize(any(GcsPath.class));
+      verify(mockGcsUtil, times(2)).create(any(GcsPath.class), anyString());
+      verifyNoMoreInteractions(mockGcsUtil);
+    }
+  }
+
+  @Test
+  public void testPackageUploadIsSkippedWhenFileAlreadyExists() throws Exception {
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
+
+    PackageUtil.stageClasspathElements(
+        ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verifyNoMoreInteractions(mockGcsUtil);
+  }
+
+  @Test
+  public void testPackageUploadIsNotSkippedWhenSizesAreDifferent() throws Exception {
+    Pipe pipe = Pipe.open();
+    File tmpDirectory = tmpFolder.newFolder("folder");
+    tmpFolder.newFolder("folder", "empty_directory");
+    tmpFolder.newFolder("folder", "directory");
+    makeFileWithContents("folder/file.txt", "This is a test!");
+    makeFileWithContents("folder/directory/file.txt", "This is also a test!");
+    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(Long.MAX_VALUE);
+    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+    PackageUtil.stageClasspathElements(
+        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+    verifyNoMoreInteractions(mockGcsUtil);
+  }
+
+  @Test
+  public void testPackageUploadWithExplicitPackageName() throws Exception {
+    Pipe pipe = Pipe.open();
+    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+    final String overriddenName = "alias.txt";
+
+    when(mockGcsUtil.fileSize(any(GcsPath.class)))
+        .thenThrow(new FileNotFoundException("some/path"));
+    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+        ImmutableList.of(overriddenName + "=" + tmpFile.getAbsolutePath()), STAGING_PATH);
+    DataflowPackage target = Iterables.getOnlyElement(targets);
+
+    verify(mockGcsUtil).fileSize(any(GcsPath.class));
+    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+    verifyNoMoreInteractions(mockGcsUtil);
+
+    assertThat(target.getName(), equalTo(overriddenName));
+    assertThat(target.getLocation(),
+        RegexMatcher.matches(STAGING_PATH + "/file-" + HASH_PATTERN + ".txt"));
+  }
+
+  @Test
+  public void testPackageUploadIsSkippedWithNonExistentResource() throws Exception {
+    String nonExistentFile =
+        IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "non-existent-file");
+    assertEquals(Collections.EMPTY_LIST, PackageUtil.stageClasspathElements(
+        ImmutableList.of(nonExistentFile), STAGING_PATH));
+  }
+
+  /**
+   * Builds a fake GoogleJsonResponseException for testing API error handling.
+   */
+  private static GoogleJsonResponseException googleJsonResponseException(
+      final int status, final String reason, final String message) throws IOException {
+    final JsonFactory jsonFactory = new JacksonFactory();
+    HttpTransport transport = new MockHttpTransport() {
+      @Override
+      public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
+        ErrorInfo errorInfo = new ErrorInfo();
+        errorInfo.setReason(reason);
+        errorInfo.setMessage(message);
+        errorInfo.setFactory(jsonFactory);
+        GenericJson error = new GenericJson();
+        error.set("code", status);
+        error.set("errors", Arrays.asList(errorInfo));
+        error.setFactory(jsonFactory);
+        GenericJson errorResponse = new GenericJson();
+        errorResponse.set("error", error);
+        errorResponse.setFactory(jsonFactory);
+        return new MockLowLevelHttpRequest().setResponse(
+            new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
+            .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
+        }
+    };
+    HttpRequest request =
+        transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
+    request.setThrowExceptionOnExecuteError(false);
+    HttpResponse response = request.execute();
+    return GoogleJsonResponseException.from(jsonFactory, response);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
deleted file mode 100644
index db20d41..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
+++ /dev/null
@@ -1,503 +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 com.google.cloud.dataflow.sdk;
-
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformHierarchy;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.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;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * A {@link Pipeline} manages a directed acyclic graph of {@link PTransform PTransforms}, and the
- * {@link PCollection PCollections} that the {@link PTransform}s consume and produce.
- *
- * <p>A {@link Pipeline} is initialized with a {@link PipelineRunner} that will later
- * execute the {@link Pipeline}.
- *
- * <p>{@link Pipeline Pipelines} are independent, so they can be constructed and executed
- * concurrently.
- *
- * <p>Each {@link Pipeline} is self-contained and isolated from any other
- * {@link Pipeline}. The {@link PValue PValues} that are inputs and outputs of each of a
- * {@link Pipeline Pipeline's} {@link PTransform PTransforms} are also owned by that
- * {@link Pipeline}. A {@link PValue} owned by one {@link Pipeline} can be read only by
- * {@link PTransform PTransforms} also owned by that {@link Pipeline}.
- *
- * <p>Here is a typical example of use:
- * <pre> {@code
- * // Start by defining the options for the pipeline.
- * PipelineOptions options = PipelineOptionsFactory.create();
- * // Then create the pipeline. The runner is determined by the options.
- * Pipeline p = Pipeline.create(options);
- *
- * // A root PTransform, like TextIO.Read or Create, gets added
- * // to the Pipeline by being applied:
- * PCollection<String> lines =
- *     p.apply(TextIO.Read.from("gs://bucket/dir/file*.txt"));
- *
- * // A Pipeline can have multiple root transforms:
- * PCollection<String> moreLines =
- *     p.apply(TextIO.Read.from("gs://bucket/other/dir/file*.txt"));
- * PCollection<String> yetMoreLines =
- *     p.apply(Create.of("yet", "more", "lines").withCoder(StringUtf8Coder.of()));
- *
- * // Further PTransforms can be applied, in an arbitrary (acyclic) graph.
- * // Subsequent PTransforms (and intermediate PCollections etc.) are
- * // implicitly part of the same Pipeline.
- * PCollection<String> allLines =
- *     PCollectionList.of(lines).and(moreLines).and(yetMoreLines)
- *     .apply(new Flatten<String>());
- * PCollection<KV<String, Integer>> wordCounts =
- *     allLines
- *     .apply(ParDo.of(new ExtractWords()))
- *     .apply(new Count<String>());
- * PCollection<String> formattedWordCounts =
- *     wordCounts.apply(ParDo.of(new FormatCounts()));
- * formattedWordCounts.apply(TextIO.Write.to("gs://bucket/dir/counts.txt"));
- *
- * // PTransforms aren't executed when they're applied, rather they're
- * // just added to the Pipeline.  Once the whole Pipeline of PTransforms
- * // is constructed, the Pipeline's PTransforms can be run using a
- * // PipelineRunner.  The default PipelineRunner executes the Pipeline
- * // directly, sequentially, in this one process, which is useful for
- * // unit tests and simple experiments:
- * p.run();
- *
- * } </pre>
- */
-public class Pipeline {
-  private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
-
-  /**
-   * Thrown during execution of a {@link Pipeline}, whenever user code within that
-   * {@link Pipeline} throws an exception.
-   *
-   * <p>The original exception thrown by user code may be retrieved via {@link #getCause}.
-   */
-  public static class PipelineExecutionException extends RuntimeException {
-    /**
-     * Wraps {@code cause} into a {@link PipelineExecutionException}.
-     */
-    public PipelineExecutionException(Throwable cause) {
-      super(cause);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Public operations.
-
-  /**
-   * Constructs a pipeline from the provided options.
-   *
-   * @return The newly created pipeline.
-   */
-  public static Pipeline create(PipelineOptions options) {
-    Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options);
-    LOG.debug("Creating {}", pipeline);
-    return pipeline;
-  }
-
-  /**
-   * Returns a {@link PBegin} owned by this Pipeline.  This is useful
-   * as the input of a root PTransform such as {@link Read} or
-   * {@link Create}.
-   */
-  public PBegin begin() {
-    return PBegin.in(this);
-  }
-
-  /**
-   * Like {@link #apply(String, PTransform)} but the transform node in the {@link Pipeline}
-   * graph will be named according to {@link PTransform#getName}.
-   *
-   * @see #apply(String, PTransform)
-   */
-  public <OutputT extends POutput> OutputT apply(
-      PTransform<? super PBegin, OutputT> root) {
-    return begin().apply(root);
-  }
-
-  /**
-   * Adds a root {@link PTransform}, such as {@link Read} or {@link Create},
-   * to this {@link Pipeline}.
-   *
-   * <p>The node in the {@link Pipeline} graph will use the provided {@code name}.
-   * This name is used in various places, including the monitoring UI, logging,
-   * and to stably identify this node in the {@link Pipeline} graph upon update.
-   *
-   * <p>Alias for {@code begin().apply(name, root)}.
-   */
-  public <OutputT extends POutput> OutputT apply(
-      String name, PTransform<? super PBegin, OutputT> root) {
-    return begin().apply(name, root);
-  }
-
-  /**
-   * Runs the {@link Pipeline} using its {@link PipelineRunner}.
-   */
-  public PipelineResult run() {
-    LOG.debug("Running {} via {}", this, runner);
-    try {
-      return runner.run(this);
-    } catch (UserCodeException e) {
-      // This serves to replace the stack with one that ends here and
-      // is caused by the caught UserCodeException, thereby splicing
-      // out all the stack frames in between the PipelineRunner itself
-      // and where the worker calls into the user's code.
-      throw new PipelineExecutionException(e.getCause());
-    }
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Below here are operations that aren't normally called by users.
-
-  /**
-   * Returns the {@link CoderRegistry} that this {@link Pipeline} uses.
-   */
-  public CoderRegistry getCoderRegistry() {
-    if (coderRegistry == null) {
-      coderRegistry = new CoderRegistry();
-      coderRegistry.registerStandardCoders();
-    }
-    return coderRegistry;
-  }
-
-  /**
-   * Sets the {@link CoderRegistry} that this {@link Pipeline} uses.
-   */
-  public void setCoderRegistry(CoderRegistry coderRegistry) {
-    this.coderRegistry = coderRegistry;
-  }
-
-  /**
-   * A {@link PipelineVisitor} can be passed into
-   * {@link Pipeline#traverseTopologically} to be called for each of the
-   * transforms and values in the {@link Pipeline}.
-   */
-  public interface PipelineVisitor {
-    /**
-     * Called for each composite transform after all topological predecessors have been visited
-     * but before any of its component transforms.
-     */
-    public void enterCompositeTransform(TransformTreeNode node);
-
-    /**
-     * Called for each composite transform after all of its component transforms and their outputs
-     * have been visited.
-     */
-    public void leaveCompositeTransform(TransformTreeNode node);
-
-    /**
-     * Called for each primitive transform after all of its topological predecessors
-     * and inputs have been visited.
-     */
-    public void visitTransform(TransformTreeNode node);
-
-    /**
-     * Called for each value after the transform that produced the value has been
-     * visited.
-     */
-    public void visitValue(PValue value, TransformTreeNode producer);
-  }
-
-  /**
-   * Invokes the {@link PipelineVisitor PipelineVisitor's}
-   * {@link PipelineVisitor#visitTransform} and
-   * {@link PipelineVisitor#visitValue} operations on each of this
-   * {@link Pipeline Pipeline's} transform and value nodes, in forward
-   * topological order.
-   *
-   * <p>Traversal of the {@link Pipeline} causes {@link PTransform PTransforms} and
-   * {@link PValue PValues} owned by the {@link Pipeline} to be marked as finished,
-   * at which point they may no longer be modified.
-   *
-   * <p>Typically invoked by {@link PipelineRunner} subclasses.
-   */
-  public void traverseTopologically(PipelineVisitor visitor) {
-    Set<PValue> visitedValues = new HashSet<>();
-    // Visit all the transforms, which should implicitly visit all the values.
-    transforms.visit(visitor, visitedValues);
-    if (!visitedValues.containsAll(values)) {
-      throw new RuntimeException(
-          "internal error: should have visited all the values "
-          + "after visiting all the transforms");
-    }
-  }
-
-  /**
-   * Like {@link #applyTransform(String, PInput, PTransform)} but defaulting to the name
-   * provided by the {@link PTransform}.
-   */
-  public static <InputT extends PInput, OutputT extends POutput>
-  OutputT applyTransform(InputT input,
-      PTransform<? super InputT, OutputT> transform) {
-    return input.getPipeline().applyInternal(transform.getName(), input, transform);
-  }
-
-  /**
-   * Applies the given {@code PTransform} to this input {@code InputT} and returns
-   * its {@code OutputT}. This uses {@code name} to identify this specific application
-   * of the transform. This name is used in various places, including the monitoring UI,
-   * logging, and to stably identify this application node in the {@link Pipeline} graph during
-   * update.
-   *
-   * <p>Each {@link PInput} subclass that provides an {@code apply} method should delegate to
-   * this method to ensure proper registration with the {@link PipelineRunner}.
-   */
-  public static <InputT extends PInput, OutputT extends POutput>
-  OutputT applyTransform(String name, InputT input,
-      PTransform<? super InputT, OutputT> transform) {
-    return input.getPipeline().applyInternal(name, input, transform);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Below here are internal operations, never called by users.
-
-  private final PipelineRunner<?> runner;
-  private final PipelineOptions options;
-  private final TransformHierarchy transforms = new TransformHierarchy();
-  private Collection<PValue> values = new ArrayList<>();
-  private Set<String> usedFullNames = new HashSet<>();
-  private CoderRegistry coderRegistry;
-  private Multimap<PTransform<?, ?>, AppliedPTransform<?, ?, ?>> transformApplicationsForTesting =
-      HashMultimap.create();
-
-  /**
-   * @deprecated replaced by {@link #Pipeline(PipelineRunner, PipelineOptions)}
-   */
-  @Deprecated
-  protected Pipeline(PipelineRunner<?> runner) {
-    this(runner, PipelineOptionsFactory.create());
-  }
-
-  protected Pipeline(PipelineRunner<?> runner, PipelineOptions options) {
-    this.runner = runner;
-    this.options = options;
-  }
-
-  @Override
-  public String toString() {
-    return "Pipeline#" + hashCode();
-  }
-
-  /**
-   * Applies a {@link PTransform} to the given {@link PInput}.
-   *
-   * @see Pipeline#apply
-   */
-  private <InputT extends PInput, OutputT extends POutput>
-  OutputT applyInternal(String name, InputT input,
-      PTransform<? super InputT, OutputT> transform) {
-    input.finishSpecifying();
-
-    TransformTreeNode parent = transforms.getCurrent();
-    String namePrefix = parent.getFullName();
-    String fullName = uniquifyInternal(namePrefix, name);
-
-    boolean nameIsUnique = fullName.equals(buildName(namePrefix, name));
-
-    if (!nameIsUnique) {
-      switch (getOptions().getStableUniqueNames()) {
-        case OFF:
-          break;
-        case WARNING:
-          LOG.warn("Transform {} does not have a stable unique name. "
-              + "This will prevent updating of pipelines.", fullName);
-          break;
-        case ERROR:
-          throw new IllegalStateException(
-              "Transform " + fullName + " does not have a stable unique name. "
-              + "This will prevent updating of pipelines.");
-        default:
-          throw new IllegalArgumentException(
-              "Unrecognized value for stable unique names: " + getOptions().getStableUniqueNames());
-      }
-    }
-
-    TransformTreeNode child =
-        new TransformTreeNode(parent, transform, fullName, input);
-    parent.addComposite(child);
-
-    transforms.addInput(child, input);
-
-    LOG.debug("Adding {} to {}", transform, this);
-    try {
-      transforms.pushNode(child);
-      transform.validate(input);
-      OutputT output = runner.apply(transform, input);
-      transforms.setOutput(child, output);
-
-      AppliedPTransform<?, ?, ?> applied = AppliedPTransform.of(
-          child.getFullName(), input, output, transform);
-      transformApplicationsForTesting.put(transform, applied);
-      // recordAsOutput is a NOOP if already called;
-      output.recordAsOutput(applied);
-      verifyOutputState(output, child);
-      return output;
-    } finally {
-      transforms.popNode();
-    }
-  }
-
-  /**
-   * Returns all producing transforms for the {@link PValue PValues} contained
-   * in {@code output}.
-   */
-  private List<AppliedPTransform<?, ?, ?>> getProducingTransforms(POutput output) {
-    List<AppliedPTransform<?, ?, ?>> producingTransforms = new ArrayList<>();
-    for (PValue value : output.expand()) {
-      AppliedPTransform<?, ?, ?> transform = value.getProducingTransformInternal();
-      if (transform != null) {
-        producingTransforms.add(transform);
-      }
-    }
-    return producingTransforms;
-  }
-
-  /**
-   * Verifies that the output of a {@link PTransform} is correctly configured in its
-   * {@link TransformTreeNode} in the {@link Pipeline} graph.
-   *
-   * <p>A non-composite {@link PTransform} must have all
-   * of its outputs registered as produced by that {@link PTransform}.
-   *
-   * <p>A composite {@link PTransform} must have all of its outputs
-   * registered as produced by the contained primitive {@link PTransform PTransforms}.
-   * They have each had the above check performed already, when
-   * they were applied, so the only possible failure state is
-   * that the composite {@link PTransform} has returned a primitive output.
-   */
-  private void verifyOutputState(POutput output, TransformTreeNode node) {
-    if (!node.isCompositeNode()) {
-      PTransform<?, ?> thisTransform = node.getTransform();
-      List<AppliedPTransform<?, ?, ?>> producingTransforms = getProducingTransforms(output);
-      for (AppliedPTransform<?, ?, ?> producingTransform : producingTransforms) {
-        // Using != because object identity indicates that the transforms
-        // are the same node in the pipeline
-        if (thisTransform != producingTransform.getTransform()) {
-          throw new IllegalArgumentException("Output of non-composite transform "
-              + thisTransform + " is registered as being produced by"
-              + " a different transform: " + producingTransform);
-        }
-      }
-    } else {
-      PTransform<?, ?> thisTransform = node.getTransform();
-      List<AppliedPTransform<?, ?, ?>> producingTransforms = getProducingTransforms(output);
-      for (AppliedPTransform<?, ?, ?> producingTransform : producingTransforms) {
-        // Using == because object identity indicates that the transforms
-        // are the same node in the pipeline
-        if (thisTransform == producingTransform.getTransform()) {
-          throw new IllegalStateException("Output of composite transform "
-              + thisTransform + " is registered as being produced by it,"
-              + " but the output of every composite transform should be"
-              + " produced by a primitive transform contained therein.");
-        }
-      }
-    }
-  }
-
-  /**
-   * Returns the configured {@link PipelineRunner}.
-   */
-  public PipelineRunner<?> getRunner() {
-    return runner;
-  }
-
-  /**
-   * Returns the configured {@link PipelineOptions}.
-   */
-  public PipelineOptions getOptions() {
-    return options;
-  }
-
-  /**
-   * @deprecated this method is no longer compatible with the design of {@link Pipeline},
-   * as {@link PTransform PTransforms} can be applied multiple times, with different names
-   * each time.
-   */
-  @Deprecated
-  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);
-    return Iterables.getOnlyElement(uses).getFullName();
-  }
-
-  /**
-   * Returns a unique name for a transform with the given prefix (from
-   * enclosing transforms) and initial name.
-   *
-   * <p>For internal use only.
-   */
-  private String uniquifyInternal(String namePrefix, String origName) {
-    String name = origName;
-    int suffixNum = 2;
-    while (true) {
-      String candidate = buildName(namePrefix, name);
-      if (usedFullNames.add(candidate)) {
-        return candidate;
-      }
-      // A duplicate!  Retry.
-      name = origName + suffixNum++;
-    }
-  }
-
-  /**
-   * Builds a name from a "/"-delimited prefix and a name.
-   */
-  private String buildName(String namePrefix, String name) {
-    return namePrefix.isEmpty() ? name : namePrefix + "/" + name;
-  }
-
-  /**
-   * Adds the given {@link PValue} to this {@link Pipeline}.
-   *
-   * <p>For internal use only.
-   */
-  public void addValueInternal(PValue value) {
-    this.values.add(value);
-    LOG.debug("Adding {} to {}", value, this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java
deleted file mode 100644
index 90d09ec..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java
+++ /dev/null
@@ -1,96 +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 com.google.cloud.dataflow.sdk;
-
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-
-/**
- * Result of {@link Pipeline#run()}.
- */
-public interface PipelineResult {
-
-  /**
-   * Retrieves the current state of the pipeline execution.
-   *
-   * @return the {@link State} representing the state of this pipeline.
-   */
-  State getState();
-
-  /**
-   * Retrieves the current value of the provided {@link Aggregator}.
-   *
-   * @param aggregator the {@link Aggregator} to retrieve values for.
-   * @return the current values of the {@link Aggregator},
-   * which may be empty if there are no values yet.
-   * @throws AggregatorRetrievalException if the {@link Aggregator} values could not be retrieved.
-   */
-  <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-      throws AggregatorRetrievalException;
-
-  // TODO: method to retrieve error messages.
-
-  /** Named constants for common values for the job state. */
-  public enum State {
-
-    /** The job state could not be obtained or was not specified. */
-    UNKNOWN(false, false),
-
-    /** The job has been paused, or has not yet started. */
-    STOPPED(false, false),
-
-    /** The job is currently running. */
-    RUNNING(false, false),
-
-    /** The job has successfully completed. */
-    DONE(true, false),
-
-    /** The job has failed. */
-    FAILED(true, false),
-
-    /** The job has been explicitly cancelled. */
-    CANCELLED(true, false),
-
-    /** The job has been updated. */
-    UPDATED(true, true);
-
-    private final boolean terminal;
-
-    private final boolean hasReplacement;
-
-    private State(boolean terminal, boolean hasReplacement) {
-      this.terminal = terminal;
-      this.hasReplacement = hasReplacement;
-    }
-
-    /**
-     * @return {@code true} if the job state can no longer complete work.
-     */
-    public final boolean isTerminal() {
-      return terminal;
-    }
-
-    /**
-     * @return {@code true} if this job state indicates that a replacement job exists.
-     */
-    public final boolean hasReplacementJob() {
-      return hasReplacement;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java
deleted file mode 100644
index adda5a2..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java
+++ /dev/null
@@ -1,52 +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 com.google.cloud.dataflow.sdk.coders;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A {@link Coder} that has no component {@link Coder Coders} or other state.
- *
- * <p>Note that, unless the behavior is overridden, atomic coders are presumed to be deterministic
- * and all instances are considered equal.
- *
- * @param <T> the type of the values being transcoded
- */
-public abstract class AtomicCoder<T> extends DeterministicStandardCoder<T> {
-  protected AtomicCoder() { }
-
-  @Override
-  public List<Coder<?>> getCoderArguments() {
-    return null;
-  }
-
-  /**
-   * Returns a list of values contained in the provided example
-   * value, one per type parameter. If there are no type parameters,
-   * returns an empty list.
-   *
-   * <p>Because {@link AtomicCoder} has no components, always returns an empty list.
-   *
-   * @param exampleValue unused, but part of the latent interface expected by
-   * {@link CoderFactories#fromStaticMethods}
-   */
-  public static <T> List<Object> getInstanceComponents(T exampleValue) {
-    return Collections.emptyList();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java
deleted file mode 100644
index e7ac0e3..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java
+++ /dev/null
@@ -1,715 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.reflect.AvroEncode;
-import org.apache.avro.reflect.AvroName;
-import org.apache.avro.reflect.AvroSchema;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.avro.reflect.Union;
-import org.apache.avro.specific.SpecificData;
-import org.apache.avro.util.ClassUtils;
-import org.apache.avro.util.Utf8;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Coder} using Avro binary format.
- *
- * <p>Each instance of {@code AvroCoder<T>} encapsulates an Avro schema for objects of type
- * {@code T}.
- *
- * <p>The Avro schema may be provided explicitly via {@link AvroCoder#of(Class, Schema)} or
- * omitted via {@link AvroCoder#of(Class)}, in which case it will be inferred
- * using Avro's {@link org.apache.avro.reflect.ReflectData}.
- *
- * <p>For complete details about schema generation and how it can be controlled please see
- * the {@link org.apache.avro.reflect} package.
- * Only concrete classes with a no-argument constructor can be mapped to Avro records.
- * All inherited fields that are not static or transient are included. Fields are not permitted to
- * be null unless annotated by {@link Nullable} or a {@link Union} schema
- * containing {@code "null"}.
- *
- * <p>To use, specify the {@code Coder} type on a PCollection:
- * <pre>
- * {@code
- * PCollection<MyCustomElement> records =
- *     input.apply(...)
- *          .setCoder(AvroCoder.of(MyCustomElement.class);
- * }
- * </pre>
- *
- * <p>or annotate the element class using {@code @DefaultCoder}.
- * <pre><code>
- * {@literal @}DefaultCoder(AvroCoder.class)
- * public class MyCustomElement {
- *   ...
- * }
- * </code></pre>
- *
- * <p>The implementation attempts to determine if the Avro encoding of the given type will satisfy
- * the criteria of {@link Coder#verifyDeterministic} by inspecting both the type and the
- * Schema provided or generated by Avro. Only coders that are deterministic can be used in
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operations.
- *
- * @param <T> the type of elements handled by this coder
- */
-public class AvroCoder<T> extends StandardCoder<T> {
-
-  /**
-   * Returns an {@code AvroCoder} instance for the provided element type.
-   * @param <T> the element type
-   */
-  public static <T> AvroCoder<T> of(TypeDescriptor<T> type) {
-    @SuppressWarnings("unchecked")
-    Class<T> clazz = (Class<T>) type.getRawType();
-    return of(clazz);
-  }
-
-  /**
-   * Returns an {@code AvroCoder} instance for the provided element class.
-   * @param <T> the element type
-   */
-  public static <T> AvroCoder<T> of(Class<T> clazz) {
-    return new AvroCoder<>(clazz, ReflectData.get().getSchema(clazz));
-  }
-
-  /**
-   * Returns an {@code AvroCoder} instance for the Avro schema. The implicit
-   * type is GenericRecord.
-   */
-  public static AvroCoder<GenericRecord> of(Schema schema) {
-    return new AvroCoder<>(GenericRecord.class, schema);
-  }
-
-  /**
-   * Returns an {@code AvroCoder} instance for the provided element type
-   * using the provided Avro schema.
-   *
-   * <p>If the type argument is GenericRecord, the schema may be arbitrary.
-   * Otherwise, the schema must correspond to the type provided.
-   *
-   * @param <T> the element type
-   */
-  public static <T> AvroCoder<T> of(Class<T> type, Schema schema) {
-    return new AvroCoder<>(type, schema);
-  }
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @JsonCreator
-  public static AvroCoder<?> of(
-      @JsonProperty("type") String classType,
-      @JsonProperty("schema") String schema) throws ClassNotFoundException {
-    Schema.Parser parser = new Schema.Parser();
-    return new AvroCoder(Class.forName(classType), parser.parse(schema));
-  }
-
-  public static final CoderProvider PROVIDER = new CoderProvider() {
-    @Override
-    public <T> Coder<T> getCoder(TypeDescriptor<T> typeDescriptor) {
-      // This is a downcast from `? super T` to T. However, because
-      // it comes from a TypeDescriptor<T>, the class object itself
-      // is the same so the supertype in question shares the same
-      // generated AvroCoder schema.
-      @SuppressWarnings("unchecked")
-      Class<T> rawType = (Class<T>) typeDescriptor.getRawType();
-      return AvroCoder.of(rawType);
-    }
-  };
-
-  private final Class<T> type;
-  private final Schema schema;
-
-  private final List<String> nonDeterministicReasons;
-
-  // Factories allocated by .get() are thread-safe and immutable.
-  private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get();
-  private static final DecoderFactory DECODER_FACTORY = DecoderFactory.get();
-  // Cache the old encoder/decoder and let the factories reuse them when possible. To be threadsafe,
-  // these are ThreadLocal. This code does not need to be re-entrant as AvroCoder does not use
-  // an inner coder.
-  private final ThreadLocal<BinaryDecoder> decoder;
-  private final ThreadLocal<BinaryEncoder> encoder;
-  private final ThreadLocal<DatumWriter<T>> writer;
-  private final ThreadLocal<DatumReader<T>> reader;
-
-  protected AvroCoder(Class<T> type, Schema schema) {
-    this.type = type;
-    this.schema = schema;
-
-    nonDeterministicReasons = new AvroDeterminismChecker().check(TypeDescriptor.of(type), schema);
-
-    // Decoder and Encoder start off null for each thread. They are allocated and potentially
-    // reused inside encode/decode.
-    this.decoder = new ThreadLocal<>();
-    this.encoder = new ThreadLocal<>();
-
-    // Reader and writer are allocated once per thread and are "final" for thread-local Coder
-    // instance.
-    this.reader = new ThreadLocal<DatumReader<T>>() {
-      @Override
-      public DatumReader<T> initialValue() {
-        return createDatumReader();
-      }
-    };
-    this.writer = new ThreadLocal<DatumWriter<T>>() {
-      @Override
-      public DatumWriter<T> initialValue() {
-        return createDatumWriter();
-      }
-    };
-  }
-
-  /**
-   * The encoding identifier is designed to support evolution as per the design of Avro
-   * In order to use this class effectively, carefully read the Avro
-   * documentation at
-   * <a href="https://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution">Schema Resolution</a>
-   * to ensure that the old and new schema <i>match</i>.
-   *
-   * <p>In particular, this encoding identifier is guaranteed to be the same for {@code AvroCoder}
-   * instances of the same principal class, and otherwise distinct. The schema is not included
-   * in the identifier.
-   *
-   * <p>When modifying a class to be encoded as Avro, here are some guidelines; see the above link
-   * for greater detail.
-   *
-   * <ul>
-   * <li>Avoid changing field names.
-   * <li>Never remove a <code>required</code> field.
-   * <li>Only add <code>optional</code> fields, with sensible defaults.
-   * <li>When changing the type of a field, consult the Avro documentation to ensure the new and
-   * old types are interchangeable.
-   * </ul>
-   *
-   * <p>Code consuming this message class should be prepared to support <i>all</i> versions of
-   * the class until it is certain that no remaining serialized instances exist.
-   *
-   * <p>If backwards incompatible changes must be made, the best recourse is to change the name
-   * of your class.
-   */
-  @Override
-  public String getEncodingId() {
-    return type.getName();
-  }
-
-  /**
-   * Returns the type this coder encodes/decodes.
-   */
-  public Class<T> getType() {
-    return type;
-  }
-
-  private Object writeReplace() {
-    // When serialized by Java, instances of AvroCoder should be replaced by
-    // a SerializedAvroCoderProxy.
-    return new SerializedAvroCoderProxy<>(type, schema.toString());
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context) throws IOException {
-    // Get a BinaryEncoder instance from the ThreadLocal cache and attempt to reuse it.
-    BinaryEncoder encoderInstance = ENCODER_FACTORY.directBinaryEncoder(outStream, encoder.get());
-    // Save the potentially-new instance for reuse later.
-    encoder.set(encoderInstance);
-    writer.get().write(value, encoderInstance);
-    // Direct binary encoder does not buffer any data and need not be flushed.
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws IOException {
-    // Get a BinaryDecoder instance from the ThreadLocal cache and attempt to reuse it.
-    BinaryDecoder decoderInstance = DECODER_FACTORY.directBinaryDecoder(inStream, decoder.get());
-    // Save the potentially-new instance for later.
-    decoder.set(decoderInstance);
-    return reader.get().read(null, decoderInstance);
-  }
-
-  @Override
-    public List<? extends Coder<?>> getCoderArguments() {
-    return null;
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    addString(result, "type", type.getName());
-    addString(result, "schema", schema.toString());
-    return result;
-  }
-
-  /**
-   * @throws NonDeterministicException when the type may not be deterministically
-   * encoded using the given {@link Schema}, the {@code directBinaryEncoder}, and the
-   * {@link ReflectDatumWriter} or {@link GenericDatumWriter}.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    if (!nonDeterministicReasons.isEmpty()) {
-      throw new NonDeterministicException(this, nonDeterministicReasons);
-    }
-  }
-
-  /**
-   * Returns a new {@link DatumReader} that can be used to read from an Avro file directly. Assumes
-   * the schema used to read is the same as the schema that was used when writing.
-   *
-   * @deprecated For {@code AvroCoder} internal use only.
-   */
-  // TODO: once we can remove this deprecated function, inline in constructor.
-  @Deprecated
-  public DatumReader<T> createDatumReader() {
-    if (type.equals(GenericRecord.class)) {
-      return new GenericDatumReader<>(schema);
-    } else {
-      return new ReflectDatumReader<>(schema);
-    }
-  }
-
-  /**
-   * Returns a new {@link DatumWriter} that can be used to write to an Avro file directly.
-   *
-   * @deprecated For {@code AvroCoder} internal use only.
-   */
-  // TODO: once we can remove this deprecated function, inline in constructor.
-  @Deprecated
-  public DatumWriter<T> createDatumWriter() {
-    if (type.equals(GenericRecord.class)) {
-      return new GenericDatumWriter<>(schema);
-    } else {
-      return new ReflectDatumWriter<>(schema);
-    }
-  }
-
-  /**
-   * Returns the schema used by this coder.
-   */
-  public Schema getSchema() {
-    return schema;
-  }
-
-  /**
-   * Proxy to use in place of serializing the {@link AvroCoder}. This allows the fields
-   * to remain final.
-   */
-  private static class SerializedAvroCoderProxy<T> implements Serializable {
-    private final Class<T> type;
-    private final String schemaStr;
-
-    public SerializedAvroCoderProxy(Class<T> type, String schemaStr) {
-      this.type = type;
-      this.schemaStr = schemaStr;
-    }
-
-    private Object readResolve() {
-      // When deserialized, instances of this object should be replaced by
-      // constructing an AvroCoder.
-      Schema.Parser parser = new Schema.Parser();
-      return new AvroCoder<T>(type, parser.parse(schemaStr));
-    }
-  }
-
-  /**
-   * Helper class encapsulating the various pieces of state maintained by the
-   * recursive walk used for checking if the encoding will be deterministic.
-   */
-  private static class AvroDeterminismChecker {
-
-    // Reasons that the original type are not deterministic. This accumulates
-    // the actual output.
-    private List<String> reasons = new ArrayList<>();
-
-    // Types that are currently "open". Used to make sure we don't have any
-    // recursive types. Note that we assume that all occurrences of a given type
-    // are equal, rather than tracking pairs of type + schema.
-    private Set<TypeDescriptor<?>> activeTypes = new HashSet<>();
-
-    // Similarly to how we record active types, we record the schemas we visit
-    // to make sure we don't encounter recursive fields.
-    private Set<Schema> activeSchemas = new HashSet<>();
-
-    /**
-     * Report an error in the current context.
-     */
-    private void reportError(String context, String fmt, Object... args) {
-      String message = String.format(fmt, args);
-      reasons.add(context + ": " + message);
-    }
-
-    /**
-     * Classes that are serialized by Avro as a String include
-     * <ul>
-     * <li>Subtypes of CharSequence (including String, Avro's mutable Utf8, etc.)
-     * <li>Several predefined classes (BigDecimal, BigInteger, URI, URL)
-     * <li>Classes annotated with @Stringable (uses their #toString() and a String constructor)
-     * </ul>
-     *
-     * <p>Rather than determine which of these cases are deterministic, we list some classes
-     * that definitely are, and treat any others as non-deterministic.
-     */
-    private static final Set<Class<?>> DETERMINISTIC_STRINGABLE_CLASSES = new HashSet<>();
-    static {
-      // CharSequences:
-      DETERMINISTIC_STRINGABLE_CLASSES.add(String.class);
-      DETERMINISTIC_STRINGABLE_CLASSES.add(Utf8.class);
-
-      // Explicitly Stringable:
-      DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigDecimal.class);
-      DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigInteger.class);
-      DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URI.class);
-      DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URL.class);
-
-      // Classes annotated with @Stringable:
-    }
-
-    /**
-     * Return true if the given type token is a subtype of *any* of the listed parents.
-     */
-    private static boolean isSubtypeOf(TypeDescriptor<?> type, Class<?>... parents) {
-      for (Class<?> parent : parents) {
-        if (type.isSubtypeOf(TypeDescriptor.of(parent))) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    protected AvroDeterminismChecker() {}
-
-    // The entry point for the check. Should not be recursively called.
-    public List<String> check(TypeDescriptor<?> type, Schema schema) {
-      recurse(type.getRawType().getName(), type, schema);
-      return reasons;
-    }
-
-    // This is the method that should be recursively called. It sets up the path
-    // and visited types correctly.
-    private void recurse(String context, TypeDescriptor<?> type, Schema schema) {
-      if (type.getRawType().isAnnotationPresent(AvroSchema.class)) {
-        reportError(context, "Custom schemas are not supported -- remove @AvroSchema.");
-        return;
-      }
-
-      if (!activeTypes.add(type)) {
-        reportError(context, "%s appears recursively", type);
-        return;
-      }
-
-      // If the the record isn't a true class, but rather a GenericRecord, SpecificRecord, etc.
-      // with a specified schema, then we need to make the decision based on the generated
-      // implementations.
-      if (isSubtypeOf(type, IndexedRecord.class)) {
-        checkIndexedRecord(context, schema, null);
-      } else {
-        doCheck(context, type, schema);
-      }
-
-      activeTypes.remove(type);
-    }
-
-    private void doCheck(String context, TypeDescriptor<?> type, Schema schema) {
-      switch (schema.getType()) {
-        case ARRAY:
-          checkArray(context, type, schema);
-          break;
-        case ENUM:
-          // Enums should be deterministic, since they depend only on the ordinal.
-          break;
-        case FIXED:
-          // Depending on the implementation of GenericFixed, we don't know how
-          // the given field will be encoded. So, we assume that it isn't
-          // deterministic.
-          reportError(context, "FIXED encodings are not guaranteed to be deterministic");
-          break;
-        case MAP:
-          checkMap(context, type, schema);
-          break;
-        case RECORD:
-          checkRecord(type, schema);
-          break;
-        case UNION:
-          checkUnion(context, type, schema);
-          break;
-        case STRING:
-          checkString(context, type);
-          break;
-        case BOOLEAN:
-        case BYTES:
-        case DOUBLE:
-        case INT:
-        case FLOAT:
-        case LONG:
-        case NULL:
-          // For types that Avro encodes using one of the above primitives, we assume they are
-          // deterministic.
-          break;
-        default:
-          // In any other case (eg., new types added to Avro) we cautiously return
-          // false.
-          reportError(context, "Unknown schema type %s may be non-deterministic", schema.getType());
-          break;
-      }
-    }
-
-    private void checkString(String context, TypeDescriptor<?> type) {
-      // For types that are encoded as strings, we need to make sure they're in an approved
-      // whitelist. For other types that are annotated @Stringable, Avro will just use the
-      // #toString() methods, which has no guarantees of determinism.
-      if (!DETERMINISTIC_STRINGABLE_CLASSES.contains(type.getRawType())) {
-        reportError(context, "%s may not have deterministic #toString()", type);
-      }
-    }
-
-   private static final Schema AVRO_NULL_SCHEMA = Schema.create(Schema.Type.NULL);
-
-   private void checkUnion(String context, TypeDescriptor<?> type, Schema schema) {
-      final List<Schema> unionTypes = schema.getTypes();
-
-      if (!type.getRawType().isAnnotationPresent(Union.class)) {
-        // First check for @Nullable field, which shows up as a union of field type and null.
-        if (unionTypes.size() == 2 && unionTypes.contains(AVRO_NULL_SCHEMA)) {
-          // Find the Schema that is not NULL and recursively check that it is deterministic.
-          Schema nullableFieldSchema = unionTypes.get(0).equals(AVRO_NULL_SCHEMA)
-              ? unionTypes.get(1) : unionTypes.get(0);
-          doCheck(context, type, nullableFieldSchema);
-          return;
-        }
-
-        // Otherwise report a schema error.
-        reportError(context, "Expected type %s to have @Union annotation", type);
-        return;
-      }
-
-      // Errors associated with this union will use the base class as their context.
-      String baseClassContext = type.getRawType().getName();
-
-      // For a union, we need to make sure that each possible instantiation is deterministic.
-      for (Schema concrete : unionTypes) {
-        @SuppressWarnings("unchecked")
-        TypeDescriptor<?> unionType = TypeDescriptor.of(ReflectData.get().getClass(concrete));
-
-        recurse(baseClassContext, unionType, concrete);
-      }
-    }
-
-    private void checkRecord(TypeDescriptor<?> type, Schema schema) {
-      // For a record, we want to make sure that all the fields are deterministic.
-      Class<?> clazz = type.getRawType();
-      for (org.apache.avro.Schema.Field fieldSchema : schema.getFields()) {
-        Field field = getField(clazz, fieldSchema.name());
-        String fieldContext = field.getDeclaringClass().getName() + "#" + field.getName();
-
-        if (field.isAnnotationPresent(AvroEncode.class)) {
-          reportError(fieldContext,
-              "Custom encoders may be non-deterministic -- remove @AvroEncode");
-          continue;
-        }
-
-        if (!IndexedRecord.class.isAssignableFrom(field.getType())
-            && field.isAnnotationPresent(AvroSchema.class)) {
-          // TODO: We should be able to support custom schemas on POJO fields, but we shouldn't
-          // need to, so we just allow it in the case of IndexedRecords.
-          reportError(fieldContext,
-              "Custom schemas are only supported for subtypes of IndexedRecord.");
-          continue;
-        }
-
-        TypeDescriptor<?> fieldType = type.resolveType(field.getGenericType());
-        recurse(fieldContext, fieldType, fieldSchema.schema());
-      }
-    }
-
-    private void checkIndexedRecord(String context, Schema schema,
-        @Nullable String specificClassStr) {
-
-      if (!activeSchemas.add(schema)) {
-        reportError(context, "%s appears recursively", schema.getName());
-        return;
-      }
-
-      switch (schema.getType()) {
-        case ARRAY:
-          // Generic Records use GenericData.Array to implement arrays, which is
-          // essentially an ArrayList, and therefore ordering is deterministic.
-          // The array is thus deterministic if the elements are deterministic.
-          checkIndexedRecord(context, schema.getElementType(), null);
-          break;
-        case ENUM:
-          // Enums are deterministic because they encode as a single integer.
-          break;
-        case FIXED:
-          // In the case of GenericRecords, FIXED is deterministic because it
-          // encodes/decodes as a Byte[].
-          break;
-        case MAP:
-          reportError(context,
-              "GenericRecord and SpecificRecords use a HashMap to represent MAPs,"
-              + " so it is non-deterministic");
-          break;
-        case RECORD:
-          for (org.apache.avro.Schema.Field field : schema.getFields()) {
-            checkIndexedRecord(
-                schema.getName() + "." + field.name(),
-                field.schema(),
-                field.getProp(SpecificData.CLASS_PROP));
-          }
-          break;
-        case STRING:
-          // GenericDatumWriter#findStringClass will use a CharSequence or a String
-          // for each string, so it is deterministic.
-
-          // SpecificCompiler#getStringType will use java.lang.String, org.apache.avro.util.Utf8,
-          // or java.lang.CharSequence, unless SpecificData.CLASS_PROP overrides that.
-          if (specificClassStr != null) {
-            Class<?> specificClass;
-            try {
-              specificClass = ClassUtils.forName(specificClassStr);
-              if (!DETERMINISTIC_STRINGABLE_CLASSES.contains(specificClass)) {
-                reportError(context, "Specific class %s is not known to be deterministic",
-                    specificClassStr);
-              }
-            } catch (ClassNotFoundException e) {
-              reportError(context, "Specific class %s is not known to be deterministic",
-                  specificClassStr);
-            }
-          }
-          break;
-        case UNION:
-          for (org.apache.avro.Schema subschema : schema.getTypes()) {
-            checkIndexedRecord(subschema.getName(), subschema, null);
-          }
-          break;
-        case BOOLEAN:
-        case BYTES:
-        case DOUBLE:
-        case INT:
-        case FLOAT:
-        case LONG:
-        case NULL:
-          // For types that Avro encodes using one of the above primitives, we assume they are
-          // deterministic.
-          break;
-        default:
-          reportError(context, "Unknown schema type %s may be non-deterministic", schema.getType());
-          break;
-      }
-
-      activeSchemas.remove(schema);
-    }
-
-    private void checkMap(String context, TypeDescriptor<?> type, Schema schema) {
-      if (!isSubtypeOf(type, SortedMap.class)) {
-        reportError(context, "%s may not be deterministically ordered", type);
-      }
-
-      // Avro (currently) asserts that all keys are strings.
-      // In case that changes, we double check that the key was a string:
-      Class<?> keyType = type.resolveType(Map.class.getTypeParameters()[0]).getRawType();
-      if (!String.class.equals(keyType)) {
-        reportError(context, "map keys should be Strings, but was %s", keyType);
-      }
-
-      recurse(context,
-          type.resolveType(Map.class.getTypeParameters()[1]),
-          schema.getValueType());
-    }
-
-    private void checkArray(String context, TypeDescriptor<?> type, Schema schema) {
-      TypeDescriptor<?> elementType = null;
-      if (type.isArray()) {
-        // The type is an array (with ordering)-> deterministic iff the element is deterministic.
-        elementType = type.getComponentType();
-      } else if (isSubtypeOf(type, Collection.class)) {
-        if (isSubtypeOf(type, List.class, SortedSet.class)) {
-          // Ordered collection -> deterministic iff the element is deterministic
-          elementType = type.resolveType(Collection.class.getTypeParameters()[0]);
-        } else {
-          // Not an ordered collection -> not deterministic
-          reportError(context, "%s may not be deterministically ordered", type);
-          return;
-        }
-      } else {
-        // If it was an unknown type encoded as an array, be conservative and assume
-        // that we don't know anything about the order.
-        reportError(context, "encoding %s as an ARRAY was unexpected");
-        return;
-      }
-
-      // If we get here, it's either a deterministically-ordered Collection, or
-      // an array. Either way, the type is deterministic iff the element type is
-      // deterministic.
-      recurse(context, elementType, schema.getElementType());
-    }
-
-    /**
-     * Extract a field from a class. We need to look at the declared fields so that we can
-     * see private fields. We may need to walk up to the parent to get classes from the parent.
-     */
-    private static Field getField(Class<?> clazz, String name) {
-      while (clazz != null) {
-        for (Field field : clazz.getDeclaredFields()) {
-          AvroName avroName = field.getAnnotation(AvroName.class);
-          if (avroName != null && name.equals(avroName.value())) {
-            return field;
-          } else if (avroName == null && name.equals(field.getName())) {
-            return field;
-          }
-        }
-        clazz = clazz.getSuperclass();
-      }
-
-      throw new IllegalArgumentException(
-          "Unable to get field " + name + " from class " + clazz);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java
deleted file mode 100644
index aea6dd5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java
+++ /dev/null
@@ -1,100 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link BigEndianIntegerCoder} encodes {@link Integer Integers} in 4 bytes, big-endian.
- */
-public class BigEndianIntegerCoder extends AtomicCoder<Integer> {
-
-  @JsonCreator
-  public static BigEndianIntegerCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final BigEndianIntegerCoder INSTANCE = new BigEndianIntegerCoder();
-
-  private BigEndianIntegerCoder() {}
-
-  @Override
-  public void encode(Integer value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    new DataOutputStream(outStream).writeInt(value);
-  }
-
-  @Override
-  public Integer decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      return new DataInputStream(inStream).readInt();
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}, because {@link #getEncodedElementByteSize} runs in constant time.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Integer value, Context context) {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code 4}, the size in bytes of an integer's big endian encoding.
-   */
-  @Override
-  protected long getEncodedElementByteSize(Integer value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    return 4;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java
deleted file mode 100644
index a0325ac..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java
+++ /dev/null
@@ -1,100 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link BigEndianLongCoder} encodes {@link Long}s in 8 bytes, big-endian.
- */
-public class BigEndianLongCoder extends AtomicCoder<Long> {
-
-  @JsonCreator
-  public static BigEndianLongCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final BigEndianLongCoder INSTANCE = new BigEndianLongCoder();
-
-  private BigEndianLongCoder() {}
-
-  @Override
-  public void encode(Long value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Long");
-    }
-    new DataOutputStream(outStream).writeLong(value);
-  }
-
-  @Override
-  public Long decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      return new DataInputStream(inStream).readLong();
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}, since {@link #getEncodedElementByteSize} returns a constant.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Long value, Context context) {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code 8}, the byte size of a big-endian encoded {@code Long}.
-   */
-  @Override
-  protected long getEncodedElementByteSize(Long value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Long");
-    }
-    return 8;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java
deleted file mode 100644
index 6b00a0e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java
+++ /dev/null
@@ -1,139 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.ExposedByteArrayOutputStream;
-import com.google.cloud.dataflow.sdk.util.StreamUtils;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.common.io.ByteStreams;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} for {@code byte[]}.
- *
- * <p>The encoding format is as follows:
- * <ul>
- * <li>If in a non-nested context (the {@code byte[]} is the only value in the stream), the
- * bytes are read/written directly.</li>
- * <li>If in a nested context, the bytes are prefixed with the length of the array,
- * encoded via a {@link VarIntCoder}.</li>
- * </ul>
- */
-public class ByteArrayCoder extends AtomicCoder<byte[]> {
-
-  @JsonCreator
-  public static ByteArrayCoder of() {
-    return INSTANCE;
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final ByteArrayCoder INSTANCE = new ByteArrayCoder();
-
-  private ByteArrayCoder() {}
-
-  @Override
-  public void encode(byte[] value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null byte[]");
-    }
-    if (!context.isWholeStream) {
-      VarInt.encode(value.length, outStream);
-      outStream.write(value);
-    } else {
-      outStream.write(value);
-    }
-  }
-
-  /**
-   * Encodes the provided {@code value} with the identical encoding to {@link #encode}, but with
-   * optimizations that take ownership of the value.
-   *
-   * <p>Once passed to this method, {@code value} should never be observed or mutated again.
-   */
-  public void encodeAndOwn(byte[] value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (!context.isWholeStream) {
-      VarInt.encode(value.length, outStream);
-      outStream.write(value);
-    } else {
-      if (outStream instanceof ExposedByteArrayOutputStream) {
-        ((ExposedByteArrayOutputStream) outStream).writeAndOwn(value);
-      } else {
-        outStream.write(value);
-      }
-    }
-  }
-
-  @Override
-  public byte[] decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    if (context.isWholeStream) {
-      return StreamUtils.getBytes(inStream);
-    } else {
-      int length = VarInt.decodeInt(inStream);
-      if (length < 0) {
-        throw new IOException("invalid length " + length);
-      }
-      byte[] value = new byte[length];
-      ByteStreams.readFully(inStream, value);
-      return value;
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return objects that are equal if the two arrays contain the same bytes.
-   */
-  @Override
-  public Object structuralValue(byte[] value) {
-    return new StructuralByteArray(value);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true} since {@link #getEncodedElementByteSize} runs in
-   * constant time using the {@code length} of the provided array.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(byte[] value, Context context) {
-    return true;
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(byte[] value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null byte[]");
-    }
-    long size = 0;
-    if (!context.isWholeStream) {
-      size += VarInt.getLength(value.length);
-    }
-    return size + value.length;
-  }
-}



[27/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java
deleted file mode 100644
index dd1b3c8..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java
+++ /dev/null
@@ -1,890 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
-import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.getString;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.core.IsInstanceOf.instanceOf;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.OutputReference;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.ArgumentMatcher;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests for DataflowPipelineTranslator.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineTranslatorTest implements Serializable {
-
-  @Rule public transient ExpectedException thrown = ExpectedException.none();
-
-  // A Custom Mockito matcher for an initial Job that checks that all
-  // expected fields are set.
-  private static class IsValidCreateRequest extends ArgumentMatcher<Job> {
-    @Override
-    public boolean matches(Object o) {
-      Job job = (Job) o;
-      return job.getId() == null
-          && job.getProjectId() == null
-          && job.getName() != null
-          && job.getType() != null
-          && job.getEnvironment() != null
-          && job.getSteps() != null
-          && job.getCurrentState() == null
-          && job.getCurrentStateTime() == null
-          && job.getExecutionInfo() == null
-          && job.getCreateTime() == null;
-    }
-  }
-
-  private DataflowPipeline buildPipeline(DataflowPipelineOptions options) {
-    DataflowPipeline p = DataflowPipeline.create(options);
-
-    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
-
-    return p;
-  }
-
-  private static Dataflow buildMockDataflow(
-      ArgumentMatcher<Job> jobMatcher) throws IOException {
-    Dataflow mockDataflowClient = mock(Dataflow.class);
-    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
-    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
-    Dataflow.Projects.Jobs.Create mockRequest = mock(
-        Dataflow.Projects.Jobs.Create.class);
-
-    when(mockDataflowClient.projects()).thenReturn(mockProjects);
-    when(mockProjects.jobs()).thenReturn(mockJobs);
-    when(mockJobs.create(eq("someProject"), argThat(jobMatcher)))
-        .thenReturn(mockRequest);
-
-    Job resultJob = new Job();
-    resultJob.setId("newid");
-    when(mockRequest.execute()).thenReturn(resultJob);
-    return mockDataflowClient;
-  }
-
-  private static DataflowPipelineOptions buildPipelineOptions() throws IOException {
-    GcsUtil mockGcsUtil = mock(GcsUtil.class);
-    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
-      @Override
-      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-      }
-    });
-    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
-    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
-
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setGcpCredential(new TestCredential());
-    options.setJobName("some-job-name");
-    options.setProject("some-project");
-    options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString());
-    options.setFilesToStage(new LinkedList<String>());
-    options.setDataflowClient(buildMockDataflow(new IsValidCreateRequest()));
-    options.setGcsUtil(mockGcsUtil);
-    return options;
-  }
-
-  @Test
-  public void testSettingOfSdkPipelineOptions() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setRunner(DataflowPipelineRunner.class);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    // Note that the contents of this materialized map may be changed by the act of reading an
-    // option, which will cause the default to get materialized whereas it would otherwise be
-    // left absent. It is permissible to simply alter this test to reflect current behavior.
-    Map<String, Object> settings = new HashMap<>();
-    settings.put("appName", "DataflowPipelineTranslatorTest");
-    settings.put("project", "some-project");
-    settings.put("pathValidatorClass", "com.google.cloud.dataflow.sdk.util.DataflowPathValidator");
-    settings.put("runner", "com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner");
-    settings.put("jobName", "some-job-name");
-    settings.put("tempLocation", "gs://somebucket/some/path");
-    settings.put("stagingLocation", "gs://somebucket/some/path/staging");
-    settings.put("stableUniqueNames", "WARNING");
-    settings.put("streaming", false);
-    settings.put("numberOfWorkerHarnessThreads", 0);
-    settings.put("experiments", null);
-
-    assertEquals(ImmutableMap.of("options", settings),
-        job.getEnvironment().getSdkPipelineOptions());
-  }
-
-  @Test
-  public void testNetworkConfig() throws IOException {
-    final String testNetwork = "test-network";
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setNetwork(testNetwork);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertEquals(testNetwork,
-        job.getEnvironment().getWorkerPools().get(0).getNetwork());
-  }
-
-  @Test
-  public void testNetworkConfigMissing() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertNull(job.getEnvironment().getWorkerPools().get(0).getNetwork());
-  }
-
-  @Test
-  public void testSubnetworkConfig() throws IOException {
-    final String testSubnetwork = "regions/REGION/subnetworks/SUBNETWORK";
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setSubnetwork(testSubnetwork);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertEquals(testSubnetwork,
-        job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
-  }
-
-  @Test
-  public void testSubnetworkConfigMissing() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertNull(job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
-  }
-
-  @Test
-  public void testScalingAlgorithmMissing() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    // Autoscaling settings are always set.
-    assertNull(
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getAlgorithm());
-    assertEquals(
-        0,
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getMaxNumWorkers()
-            .intValue());
-  }
-
-  @Test
-  public void testScalingAlgorithmNone() throws IOException {
-    final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling =
-        DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE;
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setAutoscalingAlgorithm(noScaling);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertEquals(
-        "AUTOSCALING_ALGORITHM_NONE",
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getAlgorithm());
-    assertEquals(
-        0,
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getMaxNumWorkers()
-            .intValue());
-  }
-
-  @Test
-  public void testMaxNumWorkersIsPassedWhenNoAlgorithmIsSet() throws IOException {
-    final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling = null;
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setMaxNumWorkers(42);
-    options.setAutoscalingAlgorithm(noScaling);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertNull(
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getAlgorithm());
-    assertEquals(
-        42,
-        job
-            .getEnvironment()
-            .getWorkerPools()
-            .get(0)
-            .getAutoscalingSettings()
-            .getMaxNumWorkers()
-            .intValue());
-  }
-
-  @Test
-  public void testZoneConfig() throws IOException {
-    final String testZone = "test-zone-1";
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setZone(testZone);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertEquals(testZone,
-        job.getEnvironment().getWorkerPools().get(0).getZone());
-  }
-
-  @Test
-  public void testWorkerMachineTypeConfig() throws IOException {
-    final String testMachineType = "test-machine-type";
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setWorkerMachineType(testMachineType);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-
-    WorkerPool workerPool = job.getEnvironment().getWorkerPools().get(0);
-    assertEquals(testMachineType, workerPool.getMachineType());
-  }
-
-  @Test
-  public void testDiskSizeGbConfig() throws IOException {
-    final Integer diskSizeGb = 1234;
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setDiskSizeGb(diskSizeGb);
-
-    DataflowPipeline p = buildPipeline(options);
-    p.traverseTopologically(new RecordingPipelineVisitor());
-    Job job =
-        DataflowPipelineTranslator.fromOptions(options)
-            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
-            .getJob();
-
-    assertEquals(1, job.getEnvironment().getWorkerPools().size());
-    assertEquals(diskSizeGb,
-        job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb());
-  }
-
-  @Test
-  public void testPredefinedAddStep() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-    DataflowPipelineTranslator.registerTransformTranslator(
-        EmbeddedTransform.class, new EmbeddedTranslator());
-
-    // Create a predefined step using another pipeline
-    Step predefinedStep = createPredefinedStep();
-
-    // Create a pipeline that the predefined step will be embedded into
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
-        .apply(ParDo.of(new NoOpFn()))
-        .apply(new EmbeddedTransform(predefinedStep.clone()))
-        .apply(ParDo.of(new NoOpFn()));
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(4, steps.size());
-
-    // The input to the embedded step should match the output of the step before
-    Map<String, Object> step1Out = getOutputPortReference(steps.get(1));
-    Map<String, Object> step2In = getDictionary(
-        steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT);
-    assertEquals(step1Out, step2In);
-
-    // The output from the embedded step should match the input of the step after
-    Map<String, Object> step2Out = getOutputPortReference(steps.get(2));
-    Map<String, Object> step3In = getDictionary(
-        steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT);
-    assertEquals(step2Out, step3In);
-
-    // The step should not have been modified other than remapping the input
-    Step predefinedStepClone = predefinedStep.clone();
-    Step embeddedStepClone = steps.get(2).clone();
-    predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
-    embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
-    assertEquals(predefinedStepClone, embeddedStepClone);
-  }
-
-  /**
-   * Construct a OutputReference for the output of the step.
-   */
-  private static OutputReference getOutputPortReference(Step step) throws Exception {
-    // TODO: This should be done via a Structs accessor.
-    @SuppressWarnings("unchecked")
-    List<Map<String, Object>> output =
-        (List<Map<String, Object>>) step.getProperties().get(PropertyNames.OUTPUT_INFO);
-    String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME);
-    return new OutputReference(step.getName(), outputTagId);
-  }
-
-  /**
-   * Returns a Step for a DoFn by creating and translating a pipeline.
-   */
-  private static Step createPredefinedStep() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    String stepName = "DoFn1";
-    pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
-        .apply(ParDo.of(new NoOpFn()).named(stepName))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out"));
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    assertEquals(13, job.getSteps().size());
-    Step step = job.getSteps().get(1);
-    assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME));
-    return step;
-  }
-
-  private static class NoOpFn extends DoFn<String, String> {
-    @Override public void processElement(ProcessContext c) throws Exception {
-      c.output(c.element());
-    }
-  }
-
-  /**
-   * A placeholder transform that will be used to substitute a predefined Step.
-   */
-  private static class EmbeddedTransform
-      extends PTransform<PCollection<String>, PCollection<String>> {
-    private final Step step;
-
-    public EmbeddedTransform(Step step) {
-      this.step = step;
-    }
-
-    @Override
-    public PCollection<String> apply(PCollection<String> input) {
-      return PCollection.createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          input.isBounded());
-    }
-
-    @Override
-    protected Coder<?> getDefaultOutputCoder() {
-      return StringUtf8Coder.of();
-    }
-  }
-
-  /**
-   * A TransformTranslator that adds the predefined Step using
-   * {@link TranslationContext#addStep} and remaps the input port reference.
-   */
-  private static class EmbeddedTranslator
-      implements DataflowPipelineTranslator.TransformTranslator<EmbeddedTransform> {
-    @Override public void translate(EmbeddedTransform transform, TranslationContext context) {
-      addObject(transform.step.getProperties(), PropertyNames.PARALLEL_INPUT,
-          context.asOutputReference(context.getInput(transform)));
-      context.addStep(transform, transform.step);
-    }
-  }
-
-  /**
-   * A composite transform that returns an output that is unrelated to
-   * the input.
-   */
-  private static class UnrelatedOutputCreator
-      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
-
-    @Override
-    public PCollection<Integer> apply(PCollection<Integer> input) {
-      // Apply an operation so that this is a composite transform.
-      input.apply(Count.<Integer>perElement());
-
-      // Return a value unrelated to the input.
-      return input.getPipeline().apply(Create.of(1, 2, 3, 4));
-    }
-
-    @Override
-    protected Coder<?> getDefaultOutputCoder() {
-      return VarIntCoder.of();
-    }
-  }
-
-  /**
-   * A composite transform that returns an output that is unbound.
-   */
-  private static class UnboundOutputCreator
-      extends PTransform<PCollection<Integer>, PDone> {
-
-    @Override
-    public PDone apply(PCollection<Integer> input) {
-      // Apply an operation so that this is a composite transform.
-      input.apply(Count.<Integer>perElement());
-
-      return PDone.in(input.getPipeline());
-    }
-
-    @Override
-    protected Coder<?> getDefaultOutputCoder() {
-      return VoidCoder.of();
-    }
-  }
-
-  /**
-   * A composite transform that returns a partially bound output.
-   *
-   * <p>This is not allowed and will result in a failure.
-   */
-  private static class PartiallyBoundOutputCreator
-      extends PTransform<PCollection<Integer>, PCollectionTuple> {
-
-    public final TupleTag<Integer> sumTag = new TupleTag<>("sum");
-    public final TupleTag<Void> doneTag = new TupleTag<>("done");
-
-    @Override
-    public PCollectionTuple apply(PCollection<Integer> input) {
-      PCollection<Integer> sum = input.apply(Sum.integersGlobally());
-
-      // Fails here when attempting to construct a tuple with an unbound object.
-      return PCollectionTuple.of(sumTag, sum)
-          .and(doneTag, PCollection.<Void>createPrimitiveOutputInternal(
-              input.getPipeline(),
-              WindowingStrategy.globalDefault(),
-              input.isBounded()));
-    }
-  }
-
-  @Test
-  public void testMultiGraphPipelineSerialization() throws IOException {
-    DataflowPipeline p = DataflowPipeline.create(buildPipelineOptions());
-
-    PCollection<Integer> input = p.begin()
-        .apply(Create.of(1, 2, 3));
-
-    input.apply(new UnrelatedOutputCreator());
-    input.apply(new UnboundOutputCreator());
-
-    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(
-        PipelineOptionsFactory.as(DataflowPipelineOptions.class));
-
-    // Check that translation doesn't fail.
-    t.translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList());
-  }
-
-  @Test
-  public void testPartiallyBoundFailure() throws IOException {
-    Pipeline p = DataflowPipeline.create(buildPipelineOptions());
-
-    PCollection<Integer> input = p.begin()
-        .apply(Create.of(1, 2, 3));
-
-    thrown.expect(IllegalStateException.class);
-    input.apply(new PartiallyBoundOutputCreator());
-
-    Assert.fail("Failure expected from use of partially bound output");
-  }
-
-  /**
-   * This tests a few corner cases that should not crash.
-   */
-  @Test
-  public void testGoodWildcards() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
-
-    applyRead(pipeline, "gs://bucket/foo");
-    applyRead(pipeline, "gs://bucket/foo/");
-    applyRead(pipeline, "gs://bucket/foo/*");
-    applyRead(pipeline, "gs://bucket/foo/?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]");
-    applyRead(pipeline, "gs://bucket/foo/*baz*");
-    applyRead(pipeline, "gs://bucket/foo/*baz?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
-    applyRead(pipeline, "gs://bucket/foo/baz/*");
-    applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
-    applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
-    applyRead(pipeline, "gs://bucket/foo*/baz");
-    applyRead(pipeline, "gs://bucket/foo?/baz");
-    applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
-
-    // Check that translation doesn't fail.
-    t.translate(pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList());
-  }
-
-  private void applyRead(Pipeline pipeline, String path) {
-    pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
-  }
-
-  /**
-   * Recursive wildcards are not supported.
-   * This tests "**".
-   */
-  @Test
-  public void testBadWildcardRecursive() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
-
-    pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
-
-    // Check that translation does fail.
-    thrown.expectCause(Matchers.allOf(
-        instanceOf(IllegalArgumentException.class),
-        ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage"))));
-    t.translate(pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList());
-  }
-
-  @Test
-  public void testToSingletonTranslation() throws Exception {
-    // A "change detector" test that makes sure the translation
-    // of getting a PCollectionView<T> does not change
-    // in bad ways during refactor
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setExperiments(ImmutableList.of("disable_ism_side_input"));
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    pipeline.apply(Create.of(1))
-        .apply(View.<Integer>asSingleton());
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(2, steps.size());
-
-    Step createStep = steps.get(0);
-    assertEquals("CreateCollection", createStep.getKind());
-
-    Step collectionToSingletonStep = steps.get(1);
-    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
-
-  }
-
-  @Test
-  public void testToIterableTranslation() throws Exception {
-    // A "change detector" test that makes sure the translation
-    // of getting a PCollectionView<Iterable<T>> does not change
-    // in bad ways during refactor
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setExperiments(ImmutableList.of("disable_ism_side_input"));
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    pipeline.apply(Create.of(1, 2, 3))
-        .apply(View.<Integer>asIterable());
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(2, steps.size());
-
-    Step createStep = steps.get(0);
-    assertEquals("CreateCollection", createStep.getKind());
-
-    Step collectionToSingletonStep = steps.get(1);
-    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
-  }
-
-  @Test
-  public void testToSingletonTranslationWithIsmSideInput() throws Exception {
-    // A "change detector" test that makes sure the translation
-    // of getting a PCollectionView<T> does not change
-    // in bad ways during refactor
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    pipeline.apply(Create.of(1))
-        .apply(View.<Integer>asSingleton());
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(5, steps.size());
-
-    @SuppressWarnings("unchecked")
-    List<Map<String, Object>> toIsmRecordOutputs =
-        (List<Map<String, Object>>) steps.get(3).getProperties().get(PropertyNames.OUTPUT_INFO);
-    assertTrue(
-        Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
-
-    Step collectionToSingletonStep = steps.get(4);
-    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
-  }
-
-  @Test
-  public void testToIterableTranslationWithIsmSideInput() throws Exception {
-    // A "change detector" test that makes sure the translation
-    // of getting a PCollectionView<Iterable<T>> does not change
-    // in bad ways during refactor
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-    pipeline.apply(Create.of(1, 2, 3))
-        .apply(View.<Integer>asIterable());
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(3, steps.size());
-
-    @SuppressWarnings("unchecked")
-    List<Map<String, Object>> toIsmRecordOutputs =
-        (List<Map<String, Object>>) steps.get(1).getProperties().get(PropertyNames.OUTPUT_INFO);
-    assertTrue(
-        Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
-
-
-    Step collectionToSingletonStep = steps.get(2);
-    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
-  }
-
-  @Test
-  public void testStepDisplayData() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-    DataflowPipeline pipeline = DataflowPipeline.create(options);
-
-    DoFn<Integer, Integer> fn1 = new DoFn<Integer, Integer>() {
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(c.element());
-      }
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        builder
-            .add("foo", "bar")
-            .add("foo2", DataflowPipelineTranslatorTest.class)
-                .withLabel("Test Class")
-                .withLinkUrl("http://www.google.com");
-      }
-    };
-
-    DoFn<Integer, Integer> fn2 = new DoFn<Integer, Integer>() {
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(c.element());
-      }
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        builder.add("foo3", 1234);
-      }
-    };
-
-    pipeline
-      .apply(Create.of(1, 2, 3))
-      .apply(ParDo.of(fn1))
-      .apply(ParDo.of(fn2));
-
-    Job job = translator.translate(
-        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
-
-    List<Step> steps = job.getSteps();
-    assertEquals(3, steps.size());
-
-    Map<String, Object> parDo1Properties = steps.get(1).getProperties();
-    Map<String, Object> parDo2Properties = steps.get(2).getProperties();
-    assertThat(parDo1Properties, hasKey("display_data"));
-
-    Collection<Map<String, String>> fn1displayData =
-            (Collection<Map<String, String>>) parDo1Properties.get("display_data");
-    Collection<Map<String, String>> fn2displayData =
-            (Collection<Map<String, String>>) parDo2Properties.get("display_data");
-
-    ImmutableList expectedFn1DisplayData = ImmutableList.of(
-            ImmutableMap.<String, String>builder()
-              .put("namespace", fn1.getClass().getName())
-              .put("key", "foo")
-              .put("type", "STRING")
-              .put("value", "bar")
-              .build(),
-            ImmutableMap.<String, String>builder()
-              .put("namespace", fn1.getClass().getName())
-              .put("key", "foo2")
-              .put("type", "JAVA_CLASS")
-              .put("value", DataflowPipelineTranslatorTest.class.getName())
-              .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
-              .put("label", "Test Class")
-              .put("linkUrl", "http://www.google.com")
-              .build()
-    );
-
-    ImmutableList expectedFn2DisplayData = ImmutableList.of(
-            ImmutableMap.<String, Object>builder()
-                    .put("namespace", fn2.getClass().getName())
-                    .put("key", "foo3")
-                    .put("type", "INTEGER")
-                    .put("value", 1234L)
-                    .build()
-    );
-
-    assertEquals(expectedFn1DisplayData, fn1displayData);
-    assertEquals(expectedFn2DisplayData, fn2displayData);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java
deleted file mode 100644
index 20d2bc8..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java
+++ /dev/null
@@ -1,274 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Sample;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Preconditions;
-
-import org.joda.time.Duration;
-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;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for {@link CustomSources}.
- */
-@RunWith(JUnit4.class)
-public class CustomSourcesTest {
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-  @Rule public ExpectedLogs logged = ExpectedLogs.none(CustomSources.class);
-
-  static class TestIO {
-    public static Read fromRange(int from, int to) {
-      return new Read(from, to, false);
-    }
-
-    static class Read extends BoundedSource<Integer> {
-      final int from;
-      final int to;
-      final boolean produceTimestamps;
-
-      Read(int from, int to, boolean produceTimestamps) {
-        this.from = from;
-        this.to = to;
-        this.produceTimestamps = produceTimestamps;
-      }
-
-      public Read withTimestampsMillis() {
-        return new Read(from, to, true);
-      }
-
-      @Override
-      public List<Read> splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options)
-          throws Exception {
-        List<Read> res = new ArrayList<>();
-        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-        float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers();
-        for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) {
-          res.add(new Read(
-              Math.round(from + i * step), Math.round(from + (i + 1) * step),
-              produceTimestamps));
-        }
-        return res;
-      }
-
-      @Override
-      public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
-        return 8 * (to - from);
-      }
-
-      @Override
-      public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-        return true;
-      }
-
-      @Override
-      public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
-        return new RangeReader(this);
-      }
-
-      @Override
-      public void validate() {}
-
-      @Override
-      public String toString() {
-        return "[" + from + ", " + to + ")";
-      }
-
-      @Override
-      public Coder<Integer> getDefaultOutputCoder() {
-        return BigEndianIntegerCoder.of();
-      }
-
-      private static class RangeReader extends BoundedReader<Integer> {
-        // To verify that BasicSerializableSourceFormat calls our methods according to protocol.
-        enum State {
-          UNSTARTED,
-          STARTED,
-          FINISHED
-        }
-        private Read source;
-        private int current = -1;
-        private State state = State.UNSTARTED;
-
-        public RangeReader(Read source) {
-          this.source = source;
-        }
-
-        @Override
-        public boolean start() throws IOException {
-          Preconditions.checkState(state == State.UNSTARTED);
-          state = State.STARTED;
-          current = source.from;
-          return (current < source.to);
-        }
-
-        @Override
-        public boolean advance() throws IOException {
-          Preconditions.checkState(state == State.STARTED);
-          if (current == source.to - 1) {
-            state = State.FINISHED;
-            return false;
-          }
-          current++;
-          return true;
-        }
-
-        @Override
-        public Integer getCurrent() {
-          Preconditions.checkState(state == State.STARTED);
-          return current;
-        }
-
-        @Override
-        public Instant getCurrentTimestamp() {
-          return source.produceTimestamps
-              ? new Instant(current /* as millis */) : BoundedWindow.TIMESTAMP_MIN_VALUE;
-        }
-
-        @Override
-        public void close() throws IOException {
-          Preconditions.checkState(state == State.STARTED || state == State.FINISHED);
-          state = State.FINISHED;
-        }
-
-        @Override
-        public Read getCurrentSource() {
-          return source;
-        }
-
-        @Override
-        public Read splitAtFraction(double fraction) {
-          int proposedIndex = (int) (source.from + fraction * (source.to - source.from));
-          if (proposedIndex <= current) {
-            return null;
-          }
-          Read primary = new Read(source.from, proposedIndex, source.produceTimestamps);
-          Read residual = new Read(proposedIndex, source.to, source.produceTimestamps);
-          this.source = primary;
-          return residual;
-        }
-
-        @Override
-        public Double getFractionConsumed() {
-          return (current == -1)
-              ? 0.0
-              : (1.0 * (1 + current - source.from) / (source.to - source.from));
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testDirectPipelineWithoutTimestamps() throws Exception {
-    Pipeline p = TestPipeline.create();
-    PCollection<Integer> sum = p
-        .apply(Read.from(TestIO.fromRange(10, 20)))
-        .apply(Sum.integersGlobally())
-        .apply(Sample.<Integer>any(1));
-
-    PAssert.thatSingleton(sum).isEqualTo(145);
-    p.run();
-  }
-
-  @Test
-  public void testDirectPipelineWithTimestamps() throws Exception {
-    Pipeline p = TestPipeline.create();
-    PCollection<Integer> sums =
-        p.apply(Read.from(TestIO.fromRange(10, 20).withTimestampsMillis()))
-         .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(3))))
-         .apply(Sum.integersGlobally().withoutDefaults());
-    // Should group into [10 11] [12 13 14] [15 16 17] [18 19].
-    PAssert.that(sums).containsInAnyOrder(21, 37, 39, 48);
-    p.run();
-  }
-
-  @Test
-  public void testRangeProgressAndSplitAtFraction() throws Exception {
-    // Show basic usage of getFractionConsumed and splitAtFraction.
-    // This test only tests TestIO itself, not BasicSerializableSourceFormat.
-
-    DataflowPipelineOptions options =
-        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
-    TestIO.Read source = TestIO.fromRange(10, 20);
-    try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
-      assertEquals(0, reader.getFractionConsumed().intValue());
-      assertTrue(reader.start());
-      assertEquals(0.1, reader.getFractionConsumed(), 1e-6);
-      assertTrue(reader.advance());
-      assertEquals(0.2, reader.getFractionConsumed(), 1e-6);
-      // Already past 0.0 and 0.1.
-      assertNull(reader.splitAtFraction(0.0));
-      assertNull(reader.splitAtFraction(0.1));
-
-      {
-        TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.5);
-        assertNotNull(residual);
-        TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
-        assertThat(readFromSource(primary, options), contains(10, 11, 12, 13, 14));
-        assertThat(readFromSource(residual, options), contains(15, 16, 17, 18, 19));
-      }
-
-      // Range is now [10, 15) and we are at 12.
-      {
-        TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.8); // give up 14.
-        assertNotNull(residual);
-        TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
-        assertThat(readFromSource(primary, options), contains(10, 11, 12, 13));
-        assertThat(readFromSource(residual, options), contains(14));
-      }
-
-      assertTrue(reader.advance());
-      assertEquals(12, reader.getCurrent().intValue());
-      assertTrue(reader.advance());
-      assertEquals(13, reader.getCurrent().intValue());
-      assertFalse(reader.advance());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunnerTest.java
deleted file mode 100644
index 0a78a6d..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,377 +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 com.google.cloud.dataflow.sdk.testing;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.Json;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.TimeUtil;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-import org.joda.time.Instant;
-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.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-
-/** Tests for {@link TestDataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class TestDataflowPipelineRunnerTest {
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-  @Mock private MockHttpTransport transport;
-  @Mock private MockLowLevelHttpRequest request;
-  @Mock private GcsUtil mockGcsUtil;
-
-  private TestDataflowPipelineOptions options;
-  private Dataflow service;
-
-  @Before
-  public void setUp() throws Exception {
-    MockitoAnnotations.initMocks(this);
-    when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
-    doCallRealMethod().when(request).getContentAsString();
-    service = new Dataflow(transport, Transport.getJsonFactory(), null);
-
-    options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
-    options.setAppName("TestAppName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setDataflowClient(service);
-    options.setRunner(TestDataflowPipelineRunner.class);
-    options.setPathValidatorClass(NoopPathValidator.class);
-  }
-
-  @Test
-  public void testToString() {
-    assertEquals("TestDataflowPipelineRunner#TestAppName",
-        new TestDataflowPipelineRunner(options).toString());
-  }
-
-  @Test
-  public void testRunBatchJobThatSucceeds() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    assertEquals(mockJob, runner.run(p, mockRunner));
-  }
-
-  @Test
-  public void testRunBatchJobThatFails() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.FAILED);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testBatchPipelineFailsIfException() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenAnswer(new Answer<State>() {
-          @Override
-          public State answer(InvocationOnMock invocation) {
-            JobMessage message = new JobMessage();
-            message.setMessageText("FooException");
-            message.setTime(TimeUtil.toCloudTime(Instant.now()));
-            message.setMessageImportance("JOB_MESSAGE_ERROR");
-            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
-                .process(Arrays.asList(message));
-            return State.CANCELLED;
-          }
-        });
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      assertThat(expected.getMessage(), containsString("FooException"));
-      verify(mockJob, atLeastOnce()).cancel();
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testRunStreamingJobThatSucceeds() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testRunStreamingJobThatFails() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    doReturn(State.DONE).when(job).getState();
-    assertEquals(Optional.of(true), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testCheckingForSuccessWhenPAssertFails() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    doReturn(State.DONE).when(job).getState();
-    assertEquals(Optional.of(false), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, false /* tentative */));
-    doReturn(State.RUNNING).when(job).getState();
-    assertEquals(Optional.absent(), runner.checkForSuccess(job));
-  }
-
-  private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
-      throws Exception {
-    MetricStructuredName name = new MetricStructuredName();
-    name.setName(success ? "PAssertSuccess" : "PAssertFailure");
-    name.setContext(
-        tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
-
-    MetricUpdate metric = new MetricUpdate();
-    metric.setName(name);
-    metric.setScalar(BigDecimal.ONE);
-
-    MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
-    response.setContentType(Json.MEDIA_TYPE);
-    JobMetrics jobMetrics = new JobMetrics();
-    jobMetrics.setMetrics(Lists.newArrayList(metric));
-    // N.B. Setting the factory is necessary in order to get valid JSON.
-    jobMetrics.setFactory(Transport.getJsonFactory());
-    response.setContent(jobMetrics.toPrettyString());
-    return response;
-  }
-
-  @Test
-  public void testStreamingPipelineFailsIfServiceFails() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, false /* tentative */));
-    doReturn(State.FAILED).when(job).getState();
-    assertEquals(Optional.of(false), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testStreamingPipelineFailsIfException() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenAnswer(new Answer<State>() {
-          @Override
-          public State answer(InvocationOnMock invocation) {
-            JobMessage message = new JobMessage();
-            message.setMessageText("FooException");
-            message.setTime(TimeUtil.toCloudTime(Instant.now()));
-            message.setMessageImportance("JOB_MESSAGE_ERROR");
-            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
-                .process(Arrays.asList(message));
-            return State.CANCELLED;
-          }
-        });
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      assertThat(expected.getMessage(), containsString("FooException"));
-      verify(mockJob, atLeastOnce()).cancel();
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowGroupByKeyTest.java
deleted file mode 100644
index b0b011d..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowGroupByKeyTest.java
+++ /dev/null
@@ -1,111 +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 com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.joda.time.Duration;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/** Tests for {@link GroupByKey} for the {@link DataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class DataflowGroupByKeyTest {
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  /**
-   * Create a test pipeline that uses the {@link DataflowPipelineRunner} so that {@link GroupByKey}
-   * is not expanded. This is used for verifying that even without expansion the proper errors show
-   * up.
-   */
-  private Pipeline createTestServiceRunner() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("someproject");
-    options.setStagingLocation("gs://staging");
-    options.setPathValidatorClass(NoopPathValidator.class);
-    options.setDataflowClient(null);
-    return Pipeline.create(options);
-  }
-
-  @Test
-  public void testInvalidWindowsService() {
-    Pipeline p = createTestServiceRunner();
-
-    List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
-
-    PCollection<KV<String, Integer>> input =
-        p.apply(Create.of(ungroupedPairs)
-            .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
-        .apply(Window.<KV<String, Integer>>into(
-            Sessions.withGapDuration(Duration.standardMinutes(1))));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("GroupByKey must have a valid Window merge function");
-    input
-        .apply("GroupByKey", GroupByKey.<String, Integer>create())
-        .apply("GroupByKeyAgain", GroupByKey.<String, Iterable<Integer>>create());
-  }
-
-  @Test
-  public void testGroupByKeyServiceUnbounded() {
-    Pipeline p = createTestServiceRunner();
-
-    PCollection<KV<String, Integer>> input =
-        p.apply(
-            new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
-              @Override
-              public PCollection<KV<String, Integer>> apply(PBegin input) {
-                return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
-                        input.getPipeline(),
-                        WindowingStrategy.globalDefault(),
-                        PCollection.IsBounded.UNBOUNDED)
-                    .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
-              }
-            });
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(
-        "GroupByKey cannot be applied to non-bounded PCollection in the GlobalWindow without "
-        + "a trigger. Use a Window.into or Window.triggering transform prior to GroupByKey.");
-
-    input.apply("GroupByKey", GroupByKey.<String, Integer>create());
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowViewTest.java
deleted file mode 100644
index c2a4273..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/transforms/DataflowViewTest.java
+++ /dev/null
@@ -1,206 +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 com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Duration;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link View} for a {@link DataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class DataflowViewTest {
-  @Rule
-  public transient ExpectedException thrown = ExpectedException.none();
-
-  private Pipeline createTestBatchRunner() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("someproject");
-    options.setStagingLocation("gs://staging");
-    options.setPathValidatorClass(NoopPathValidator.class);
-    options.setDataflowClient(null);
-    return Pipeline.create(options);
-  }
-
-  private Pipeline createTestStreamingRunner() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setStreaming(true);
-    options.setProject("someproject");
-    options.setStagingLocation("gs://staging");
-    options.setPathValidatorClass(NoopPathValidator.class);
-    options.setDataflowClient(null);
-    return Pipeline.create(options);
-  }
-
-  private void testViewUnbounded(
-      Pipeline pipeline,
-      PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("Unable to create a side-input view from input");
-    thrown.expectCause(
-        ThrowableMessageMatcher.hasMessage(Matchers.containsString("non-bounded PCollection")));
-    pipeline
-        .apply(
-            new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
-              @Override
-              public PCollection<KV<String, Integer>> apply(PBegin input) {
-                return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
-                        input.getPipeline(),
-                        WindowingStrategy.globalDefault(),
-                        PCollection.IsBounded.UNBOUNDED)
-                    .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
-              }
-            })
-        .apply(view);
-  }
-
-  private void testViewNonmerging(
-      Pipeline pipeline,
-      PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("Unable to create a side-input view from input");
-    thrown.expectCause(
-        ThrowableMessageMatcher.hasMessage(Matchers.containsString("Consumed by GroupByKey")));
-    pipeline.apply(Create.<KV<String, Integer>>of(KV.of("hello", 5)))
-        .apply(Window.<KV<String, Integer>>into(new InvalidWindows<>(
-            "Consumed by GroupByKey", FixedWindows.of(Duration.standardHours(1)))))
-        .apply(view);
-  }
-
-  @Test
-  public void testViewUnboundedAsSingletonBatch() {
-    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
-  }
-
-  @Test
-  public void testViewUnboundedAsSingletonStreaming() {
-    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
-  }
-
-  @Test
-  public void testViewUnboundedAsIterableBatch() {
-    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
-  }
-
-  @Test
-  public void testViewUnboundedAsIterableStreaming() {
-    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
-  }
-
-  @Test
-  public void testViewUnboundedAsListBatch() {
-    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asList());
-  }
-
-  @Test
-  public void testViewUnboundedAsListStreaming() {
-    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
-  }
-
-  @Test
-  public void testViewUnboundedAsMapBatch() {
-    testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMap());
-  }
-
-  @Test
-  public void testViewUnboundedAsMapStreaming() {
-    testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMap());
-  }
-
-  @Test
-  public void testViewUnboundedAsMultimapBatch() {
-    testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMultimap());
-  }
-
-  @Test
-  public void testViewUnboundedAsMultimapStreaming() {
-    testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMultimap());
-  }
-
-  @Test
-  public void testViewNonmergingAsSingletonBatch() {
-    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
-  }
-
-  @Test
-  public void testViewNonmergingAsSingletonStreaming() {
-    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
-  }
-
-  @Test
-  public void testViewNonmergingAsIterableBatch() {
-    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
-  }
-
-  @Test
-  public void testViewNonmergingAsIterableStreaming() {
-    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
-  }
-
-  @Test
-  public void testViewNonmergingAsListBatch() {
-    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asList());
-  }
-
-  @Test
-  public void testViewNonmergingAsListStreaming() {
-    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
-  }
-
-  @Test
-  public void testViewNonmergingAsMapBatch() {
-    testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMap());
-  }
-
-  @Test
-  public void testViewNonmergingAsMapStreaming() {
-    testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMap());
-  }
-
-  @Test
-  public void testViewNonmergingAsMultimapBatch() {
-    testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMultimap());
-  }
-
-  @Test
-  public void testViewNonmergingAsMultimapStreaming() {
-    testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMultimap());
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidatorTest.java
deleted file mode 100644
index 6a3cce7..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidatorTest.java
+++ /dev/null
@@ -1,93 +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 com.google.cloud.dataflow.sdk.util;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.when;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-
-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;
-
-/** Tests for {@link DataflowPathValidator}. */
-@RunWith(JUnit4.class)
-public class DataflowPathValidatorTest {
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-
-  @Mock private GcsUtil mockGcsUtil;
-  private DataflowPathValidator validator;
-
-  @Before
-  public void setUp() throws Exception {
-    MockitoAnnotations.initMocks(this);
-    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
-    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setGcpCredential(new TestCredential());
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setGcsUtil(mockGcsUtil);
-    validator = new DataflowPathValidator(options);
-  }
-
-  @Test
-  public void testValidFilePattern() {
-    validator.validateInputFilePatternSupported("gs://bucket/path");
-  }
-
-  @Test
-  public void testInvalidFilePattern() {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
-    validator.validateInputFilePatternSupported("/local/path");
-  }
-
-  @Test
-  public void testWhenBucketDoesNotExist() throws Exception {
-    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(false);
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "Could not find file gs://non-existent-bucket/location");
-    validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
-  }
-
-  @Test
-  public void testValidOutputPrefix() {
-    validator.validateOutputFilePrefixSupported("gs://bucket/path");
-  }
-
-  @Test
-  public void testInvalidOutputPrefix() {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage(
-        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
-    validator.validateOutputFilePrefixSupported("/local/path");
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java
deleted file mode 100644
index 23e12de..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java
+++ /dev/null
@@ -1,148 +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 com.google.cloud.dataflow.sdk.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-
-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;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for MonitoringUtil.
- */
-@RunWith(JUnit4.class)
-public class MonitoringUtilTest {
-  private static final String PROJECT_ID = "someProject";
-  private static final String JOB_ID = "1234";
-
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testGetJobMessages() throws IOException {
-    Dataflow.Projects.Jobs.Messages mockMessages = mock(Dataflow.Projects.Jobs.Messages.class);
-
-    // Two requests are needed to get all the messages.
-    Dataflow.Projects.Jobs.Messages.List firstRequest =
-        mock(Dataflow.Projects.Jobs.Messages.List.class);
-    Dataflow.Projects.Jobs.Messages.List secondRequest =
-        mock(Dataflow.Projects.Jobs.Messages.List.class);
-
-    when(mockMessages.list(PROJECT_ID, JOB_ID)).thenReturn(firstRequest).thenReturn(secondRequest);
-
-    ListJobMessagesResponse firstResponse = new ListJobMessagesResponse();
-    firstResponse.setJobMessages(new ArrayList<JobMessage>());
-    for (int i = 0; i < 100; ++i) {
-      JobMessage message = new JobMessage();
-      message.setId("message_" + i);
-      message.setTime(TimeUtil.toCloudTime(new Instant(i)));
-      firstResponse.getJobMessages().add(message);
-    }
-    String pageToken = "page_token";
-    firstResponse.setNextPageToken(pageToken);
-
-    ListJobMessagesResponse secondResponse = new ListJobMessagesResponse();
-    secondResponse.setJobMessages(new ArrayList<JobMessage>());
-    for (int i = 100; i < 150; ++i) {
-      JobMessage message = new JobMessage();
-      message.setId("message_" + i);
-      message.setTime(TimeUtil.toCloudTime(new Instant(i)));
-      secondResponse.getJobMessages().add(message);
-    }
-
-    when(firstRequest.execute()).thenReturn(firstResponse);
-    when(secondRequest.execute()).thenReturn(secondResponse);
-
-    MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages);
-
-    List<JobMessage> messages = util.getJobMessages(JOB_ID, -1);
-
-    verify(secondRequest).setPageToken(pageToken);
-
-    assertEquals(150, messages.size());
-  }
-
-  @Test
-  public void testToStateCreatesState() {
-    String stateName = "JOB_STATE_DONE";
-
-    State result = MonitoringUtil.toState(stateName);
-
-    assertEquals(State.DONE, result);
-  }
-
-  @Test
-  public void testToStateWithNullReturnsUnknown() {
-    String stateName = null;
-
-    State result = MonitoringUtil.toState(stateName);
-
-    assertEquals(State.UNKNOWN, result);
-  }
-
-  @Test
-  public void testToStateWithOtherValueReturnsUnknown() {
-    String stateName = "FOO_BAR_BAZ";
-
-    State result = MonitoringUtil.toState(stateName);
-
-    assertEquals(State.UNKNOWN, result);
-  }
-
-  @Test
-  public void testDontOverrideEndpointWithDefaultApi() {
-    DataflowPipelineOptions options =
-        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
-    options.setProject(PROJECT_ID);
-    options.setGcpCredential(new TestCredential());
-    String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
-    assertEquals("gcloud alpha dataflow jobs --project=someProject cancel 1234", cancelCommand);
-  }
-
-  @Test
-  public void testOverridesEndpointWithStagedDataflowEndpoint() {
-    DataflowPipelineOptions options =
-        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
-    options.setProject(PROJECT_ID);
-    options.setGcpCredential(new TestCredential());
-    String stagingDataflowEndpoint = "v0neverExisted";
-    options.setDataflowEndpoint(stagingDataflowEndpoint);
-    String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
-    assertEquals(
-        "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW=https://dataflow.googleapis.com/v0neverExisted/ "
-        + "gcloud alpha dataflow jobs --project=someProject cancel 1234",
-        cancelCommand);
-  }
-}



[56/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/OffsetRangeTrackerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/OffsetRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/OffsetRangeTrackerTest.java
index 828841b..3de04f7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/OffsetRangeTrackerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/OffsetRangeTrackerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/user.avsc
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/user.avsc b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/user.avsc
index 0cd9cee..aceff38 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/user.avsc
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/user.avsc
@@ -1,5 +1,5 @@
 {
-  "namespace": "com.google.cloud.dataflow.sdk.io",
+  "namespace": "org.apache.beam.sdk.io",
   "type": "record",
   "name": "AvroGeneratedUser",
   "fields": [

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
index 09cd25b..c179738 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.options.GcpOptions.DefaultProjectFactory;
-import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties;
+import org.apache.beam.sdk.options.GcpOptions.DefaultProjectFactory;
+import org.apache.beam.sdk.testing.RestoreSystemProperties;
+
 import com.google.common.collect.ImmutableMap;
 import com.google.common.io.Files;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
index c2f0bb8..546fe7d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
+import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.Transport;
+
 import com.google.api.services.bigquery.Bigquery.Datasets.Delete;
 import com.google.api.services.storage.Storage;
-import com.google.cloud.dataflow.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.Transport;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 2672a39..62c6909 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.hasItem;
@@ -27,12 +27,13 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.RestoreSystemProperties;
+
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -118,7 +119,7 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
         "Expected getter for property [object] of type [java.lang.Object] on "
-        + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingGetter].");
+        + "[org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MissingGetter].");
 
     PipelineOptionsFactory.as(MissingGetter.class);
   }
@@ -132,7 +133,7 @@ public class PipelineOptionsFactoryTest {
   public void testMultipleMissingGettersThrows() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "missing property methods on [com.google.cloud.dataflow.sdk.options."
+        "missing property methods on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MissingMultipleGetters]");
     expectedException.expectMessage("getter for property [object] of type [java.lang.Object]");
     expectedException.expectMessage("getter for property [otherObject] of type [java.lang.Object]");
@@ -150,7 +151,7 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
         "Expected setter for property [object] of type [java.lang.Object] on "
-        + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter].");
+        + "[org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MissingSetter].");
 
     PipelineOptionsFactory.as(MissingSetter.class);
   }
@@ -164,7 +165,7 @@ public class PipelineOptionsFactoryTest {
   public void testMissingMultipleSettersThrows() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "missing property methods on [com.google.cloud.dataflow.sdk.options."
+        "missing property methods on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MissingMultipleSetters]");
     expectedException.expectMessage("setter for property [object] of type [java.lang.Object]");
     expectedException.expectMessage("setter for property [otherObject] of type [java.lang.Object]");
@@ -181,7 +182,7 @@ public class PipelineOptionsFactoryTest {
   public void testMissingGettersAndSettersThrows() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "missing property methods on [com.google.cloud.dataflow.sdk.options."
+        "missing property methods on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MissingGettersAndSetters]");
     expectedException.expectMessage("getter for property [object] of type [java.lang.Object]");
     expectedException.expectMessage("setter for property [otherObject] of type [java.lang.Object]");
@@ -242,7 +243,7 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
         "Methods [extraneousMethod(int, String)] on "
-        + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ExtraneousMethod] "
+        + "[org.apache.beam.sdk.options.PipelineOptionsFactoryTest$ExtraneousMethod] "
         + "do not conform to being bean properties.");
 
     PipelineOptionsFactory.as(ExtraneousMethod.class);
@@ -260,11 +261,11 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
         "Method [getObject] has multiple definitions [public abstract java.lang.Object "
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter"
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MissingSetter"
         + ".getObject(), public abstract java.lang.String "
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict"
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict"
         + ".getObject()] with different return types for ["
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict].");
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict].");
     PipelineOptionsFactory.as(ReturnTypeConflict.class);
   }
 
@@ -288,23 +289,23 @@ public class PipelineOptionsFactoryTest {
   @Test
   public void testMultipleReturnTypeConflictsThrows() throws Exception {
     expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("[com.google.cloud.dataflow.sdk.options."
+    expectedException.expectMessage("[org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MultiReturnTypeConflict]");
     expectedException.expectMessage(
         "Methods with multiple definitions with different return types");
     expectedException.expectMessage("Method [getObject] has multiple definitions");
     expectedException.expectMessage("public abstract java.lang.Object "
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$"
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$"
         + "MissingSetter.getObject()");
     expectedException.expectMessage(
-        "public abstract java.lang.String com.google.cloud.dataflow.sdk.options."
+        "public abstract java.lang.String org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MultiReturnTypeConflict.getObject()");
     expectedException.expectMessage("Method [getOther] has multiple definitions");
     expectedException.expectMessage("public abstract java.lang.Object "
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$"
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$"
         + "MultiReturnTypeConflictBase.getOther()");
     expectedException.expectMessage(
-        "public abstract java.lang.Long com.google.cloud.dataflow.sdk.options."
+        "public abstract java.lang.Long org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MultiReturnTypeConflict.getOther()");
 
     PipelineOptionsFactory.as(MultiReturnTypeConflict.class);
@@ -321,8 +322,8 @@ public class PipelineOptionsFactoryTest {
   public void testSetterAnnotatedWithJsonIgnore() throws Exception {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "Expected setter for property [value] to not be marked with @JsonIgnore on [com."
-        + "google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]");
+        "Expected setter for property [value] to not be marked with @JsonIgnore on ["
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]");
     PipelineOptionsFactory.as(SetterWithJsonIgnore.class);
   }
 
@@ -338,12 +339,11 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("Found setters marked with @JsonIgnore:");
     expectedException.expectMessage(
-        "property [other] should not be marked with @JsonIgnore on [com"
-        + ".google.cloud.dataflow.sdk.options."
-        + "PipelineOptionsFactoryTest$MultiSetterWithJsonIgnore]");
+        "property [other] should not be marked with @JsonIgnore on ["
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiSetterWithJsonIgnore]");
     expectedException.expectMessage(
-        "property [value] should not be marked with @JsonIgnore on [com."
-        + "google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]");
+        "property [value] should not be marked with @JsonIgnore on ["
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]");
     PipelineOptionsFactory.as(MultiSetterWithJsonIgnore.class);
   }
 
@@ -365,9 +365,9 @@ public class PipelineOptionsFactoryTest {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
         "Expected getter for property [object] to be marked with @JsonIgnore on all ["
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$GetterWithJsonIgnore, "
-        + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter], "
-        + "found only on [com.google.cloud.dataflow.sdk.options."
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$GetterWithJsonIgnore, "
+        + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MissingSetter], "
+        + "found only on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$GetterWithJsonIgnore]");
 
     // When we attempt to convert, we should error at this moment.
@@ -407,21 +407,21 @@ public class PipelineOptionsFactoryTest {
     expectedException.expectMessage("Property getters are inconsistently marked with @JsonIgnore:");
     expectedException.expectMessage(
         "property [object] to be marked on all");
-    expectedException.expectMessage("found only on [com.google.cloud.dataflow.sdk.options."
+    expectedException.expectMessage("found only on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MultiGetters]");
     expectedException.expectMessage(
         "property [other] to be marked on all");
-    expectedException.expectMessage("found only on [com.google.cloud.dataflow.sdk.options."
+    expectedException.expectMessage("found only on [org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore]");
 
     expectedException.expectMessage(Matchers.anyOf(
         containsString(java.util.Arrays.toString(new String[]
-            {"com.google.cloud.dataflow.sdk.options."
+            {"org.apache.beam.sdk.options."
                 + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore",
-                "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MultiGetters"})),
+                "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiGetters"})),
         containsString(java.util.Arrays.toString(new String[]
-            {"com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MultiGetters",
-                "com.google.cloud.dataflow.sdk.options."
+            {"org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiGetters",
+                "org.apache.beam.sdk.options."
                 + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore"}))));
     expectedException.expectMessage(not(containsString("property [consistent]")));
 
@@ -971,7 +971,7 @@ public class PipelineOptionsFactoryTest {
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
     assertThat(output, containsString("The set of registered options are:"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
     assertThat(output, containsString("Use --help=<OptionsName> for detailed help."));
   }
 
@@ -979,11 +979,11 @@ public class PipelineOptionsFactoryTest {
   public void testSpecificHelpAsArgument() {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     ListMultimap<String, String> arguments = ArrayListMultimap.create();
-    arguments.put("help", "com.google.cloud.dataflow.sdk.options.PipelineOptions");
+    arguments.put("help", "org.apache.beam.sdk.options.PipelineOptions");
     assertTrue(PipelineOptionsFactory.printHelpUsageAndExitIfNeeded(
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
     assertThat(output, containsString("--runner"));
     assertThat(output, containsString("Default: " + DEFAULT_RUNNER_CLASS.getSimpleName()));
     assertThat(output,
@@ -998,7 +998,7 @@ public class PipelineOptionsFactoryTest {
     assertTrue(PipelineOptionsFactory.printHelpUsageAndExitIfNeeded(
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
     assertThat(output, containsString("--runner"));
     assertThat(output, containsString("Default: " + DEFAULT_RUNNER_CLASS.getSimpleName()));
     assertThat(output,
@@ -1013,7 +1013,7 @@ public class PipelineOptionsFactoryTest {
     assertTrue(PipelineOptionsFactory.printHelpUsageAndExitIfNeeded(
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
     assertThat(output, containsString("--runner"));
     assertThat(output, containsString("Default: " + DEFAULT_RUNNER_CLASS.getSimpleName()));
     assertThat(output,
@@ -1045,12 +1045,12 @@ public class PipelineOptionsFactoryTest {
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
     assertThat(output, containsString("Multiple matches found for NameConflict"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options."
+    assertThat(output, containsString("org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$NameConflictClassA$NameConflict"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options."
+    assertThat(output, containsString("org.apache.beam.sdk.options."
         + "PipelineOptionsFactoryTest$NameConflictClassB$NameConflict"));
     assertThat(output, containsString("The set of registered options are:"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
   }
 
   @Test
@@ -1068,27 +1068,27 @@ public class PipelineOptionsFactoryTest {
   public void testHelpWithBadOptionNameAsArgument() {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     ListMultimap<String, String> arguments = ArrayListMultimap.create();
-    arguments.put("help", "com.google.cloud.dataflow.sdk.Pipeline");
+    arguments.put("help", "org.apache.beam.sdk.Pipeline");
     assertTrue(PipelineOptionsFactory.printHelpUsageAndExitIfNeeded(
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
     assertThat(output,
-        containsString("Unable to find option com.google.cloud.dataflow.sdk.Pipeline"));
+        containsString("Unable to find option org.apache.beam.sdk.Pipeline"));
     assertThat(output, containsString("The set of registered options are:"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
   }
 
   @Test
   public void testHelpWithHiddenMethodAndInterface() {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     ListMultimap<String, String> arguments = ArrayListMultimap.create();
-    arguments.put("help", "com.google.cloud.dataflow.sdk.option.DataflowPipelineOptions");
+    arguments.put("help", "org.apache.beam.sdk.option.DataflowPipelineOptions");
     assertTrue(PipelineOptionsFactory.printHelpUsageAndExitIfNeeded(
         arguments, new PrintStream(baos), false /* exit */));
     String output = new String(baos.toByteArray());
     // A hidden interface.
     assertThat(output, not(
-        containsString("com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions")));
+        containsString("org.apache.beam.sdk.options.DataflowPipelineDebugOptions")));
     // A hidden option.
     assertThat(output, not(containsString("--gcpCredential")));
   }
@@ -1099,7 +1099,7 @@ public class PipelineOptionsFactoryTest {
     PipelineOptionsFactory.printHelp(new PrintStream(baos));
     String output = new String(baos.toByteArray());
     assertThat(output, containsString("The set of registered options are:"));
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
   }
 
   @Test
@@ -1107,7 +1107,7 @@ public class PipelineOptionsFactoryTest {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PipelineOptionsFactory.printHelp(new PrintStream(baos), PipelineOptions.class);
     String output = new String(baos.toByteArray());
-    assertThat(output, containsString("com.google.cloud.dataflow.sdk.options.PipelineOptions"));
+    assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions"));
     assertThat(output, containsString("--runner"));
     assertThat(output, containsString("Default: " + DEFAULT_RUNNER_CLASS.getSimpleName()));
     assertThat(output,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java
index 83b1c9c..dfda528 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -24,7 +24,8 @@ import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
index ca23737..0250bd1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,7 +48,7 @@ public class PipelineOptionsValidatorTest {
   public void testWhenRequiredOptionIsSetAndCleared() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("Missing required value for "
-        + "[public abstract java.lang.String com.google.cloud.dataflow."
+        + "[public abstract java.lang.String org.apache.beam."
         + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"].");
 
     Required required = PipelineOptionsFactory.as(Required.class);
@@ -61,7 +61,7 @@ public class PipelineOptionsValidatorTest {
   public void testWhenRequiredOptionIsNeverSet() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("Missing required value for "
-        + "[public abstract java.lang.String com.google.cloud.dataflow."
+        + "[public abstract java.lang.String org.apache.beam."
         + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"].");
 
     Required required = PipelineOptionsFactory.as(Required.class);
@@ -72,7 +72,7 @@ public class PipelineOptionsValidatorTest {
   public void testWhenRequiredOptionIsNeverSetOnSuperInterface() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("Missing required value for "
-        + "[public abstract java.lang.String com.google.cloud.dataflow."
+        + "[public abstract java.lang.String org.apache.beam."
         + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"].");
 
     PipelineOptions options = PipelineOptionsFactory.create();
@@ -91,7 +91,7 @@ public class PipelineOptionsValidatorTest {
   public void testValidationOnOverriddenMethods() throws Exception {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("Missing required value for "
-        + "[public abstract java.lang.String com.google.cloud.dataflow."
+        + "[public abstract java.lang.String org.apache.beam."
         + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"].");
 
     SubClassValidation required = PipelineOptionsFactory.as(SubClassValidation.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index f960d26..7f0fa14 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -272,8 +272,8 @@ public class ProxyInvocationHandlerTest {
   @Test
   public void testInvokeWithUnknownMethod() throws Exception {
     expectedException.expect(RuntimeException.class);
-    expectedException.expectMessage("Unknown method [public abstract void com.google.cloud."
-        + "dataflow.sdk.options.ProxyInvocationHandlerTest$UnknownMethod.unknownMethod()] "
+    expectedException.expectMessage("Unknown method [public abstract void "
+        + "org.apache.beam.sdk.options.ProxyInvocationHandlerTest$UnknownMethod.unknownMethod()] "
         + "invoked with args [null].");
 
     ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.<String, Object>newHashMap());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractorTest.java
index 38406c7..7950a9e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -23,16 +23,17 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+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.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
index 56f7d30..92c4835 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
index b131397..2f5272b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.isA;
@@ -24,21 +24,22 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AtomicCoder;
+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.io.AvroIO;
+import org.apache.beam.sdk.io.ShardNameTemplate;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.IOChannelUtils;
+
 import com.google.common.collect.Iterables;
 import com.google.common.io.Files;
 
@@ -81,7 +82,7 @@ public class DirectPipelineRunnerTest implements Serializable {
 
     @Override
     public T decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
             throws CoderException {
       throw new CoderException("Called CrashingCoder.decode");
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineTest.java
index 6f72d89..9829ebd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index d072e62..9313439 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.TestCredential;
 
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 6945503..d926ac5 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.not;
@@ -24,22 +24,22 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.Write;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Sample;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.Write;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Sample;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PValue;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java
index 207734a..a985a31 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
 
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.DelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.DelegateCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.KV;
 
 import org.joda.time.Instant;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
index a90ba7b..43367dd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.theInstance;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.AvroIOTest;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.AvroIOTest;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
index b31e6cf..146dd98 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -24,21 +24,22 @@ import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader;
-import com.google.cloud.dataflow.sdk.io.CountingSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
index 905f58f..aef4845 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
@@ -15,26 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.emptyIterable;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.io.CountingInput;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
 
 import org.hamcrest.Matchers;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
index 726c53b..fa0cb19 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.isA;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Instant;
 import org.junit.Rule;
@@ -167,14 +167,14 @@ public class EncodabilityEnforcementFactoryTest {
     public void encode(
         Record value,
         OutputStream outStream,
-        com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       throw new CoderException("Encode not allowed");
     }
 
     @Override
     public Record decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       return null;
     }
@@ -185,12 +185,12 @@ public class EncodabilityEnforcementFactoryTest {
     public void encode(
         Record value,
         OutputStream outStream,
-        com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {}
 
     @Override
     public Record decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       throw new CoderException("Decode not allowed");
     }
@@ -201,12 +201,12 @@ public class EncodabilityEnforcementFactoryTest {
     public void encode(
         Record value,
         OutputStream outStream,
-        com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {}
 
     @Override
     public Record decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       return new Record() {
         @Override
@@ -233,12 +233,12 @@ public class EncodabilityEnforcementFactoryTest {
     public void encode(
         Record value,
         OutputStream outStream,
-        com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {}
 
     @Override
     public Record decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       return new Record() {
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
index f93abd8..5c1da14 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -23,16 +23,16 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
index 13eb745..ca3753c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
@@ -23,9 +23,9 @@ import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
index f37980c..b7ce169 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
@@ -15,25 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.contains;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItem;
-import com.google.cloud.dataflow.sdk.util.KeyedWorkItems;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Multiset;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
index 40b1d5a..386eacc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
@@ -15,25 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Instant;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
index 1739534..4520504 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Instant;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
index 93d2a42..077c0e7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -25,30 +25,31 @@ import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.BaseMatcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
index 060d43c..50122aa 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
@@ -23,14 +23,15 @@ import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleTest.java
index e576694..103ace5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
@@ -23,11 +23,12 @@ import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessCreateTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessCreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessCreateTest.java
index 8bb7e51..5c63af1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessCreateTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessCreateTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasSize;
@@ -23,25 +23,26 @@ import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.NullableCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessCreate.InMemorySource;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.inprocess.InProcessCreate.InMemorySource;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.RunnableOnService;
+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.util.SerializableUtils;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;
@@ -145,14 +146,14 @@ public class InProcessCreateTest {
       public void encode(
           UnserializableRecord value,
           OutputStream outStream,
-          com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+          org.apache.beam.sdk.coders.Coder.Context context)
           throws CoderException, IOException {
         stringCoder.encode(value.myString, outStream, context.nested());
       }
 
       @Override
       public UnserializableRecord decode(
-          InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+          InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
           throws CoderException, IOException {
         return new UnserializableRecord(stringCoder.decode(inStream, context.nested()));
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
index e1faf1b..6736562 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -25,42 +25,43 @@ import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.io.CountingInput;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.BagState;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+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.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
index 2d60052..59a96ed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessRegistrar.InProcessRunner;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+import org.apache.beam.sdk.runners.inprocess.InProcessRegistrar.InProcessRunner;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 


[12/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java
deleted file mode 100644
index eae5e8b..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java
+++ /dev/null
@@ -1,544 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.JAXBCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.common.base.Preconditions;
-
-import org.codehaus.stax2.XMLInputFactory2;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.SequenceInputStream;
-import java.nio.ByteBuffer;
-import java.nio.CharBuffer;
-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;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.ValidationEvent;
-import javax.xml.bind.ValidationEventHandler;
-import javax.xml.stream.FactoryConfigurationError;
-import javax.xml.stream.XMLInputFactory;
-import javax.xml.stream.XMLStreamConstants;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamReader;
-
-// CHECKSTYLE.OFF: JavadocStyle
-/**
- * A source that can be used to read XML files. This source reads one or more
- * XML files and creates a {@code PCollection} of a given type. An Dataflow read transform can be
- * created by passing an {@code XmlSource} object to {@code Read.from()}. Please note the
- * example given below.
- *
- * <p>The XML file must be of the following form, where {@code root} and {@code record} are XML
- * element names that are defined by the user:
- *
- * <pre>
- * {@code
- * <root>
- * <record> ... </record>
- * <record> ... </record>
- * <record> ... </record>
- * ...
- * <record> ... </record>
- * </root>
- * }
- * </pre>
- *
- * <p>Basically, the XML document should contain a single root element with an inner list consisting
- * entirely of record elements. The records may contain arbitrary XML content; however, that content
- * <b>must not</b> contain the start {@code <record>} or end {@code </record>} tags. This
- * restriction enables reading from large XML files in parallel from different offsets in the file.
- *
- * <p>Root and/or record elements may additionally contain an arbitrary number of XML attributes.
- * Additionally users must provide a class of a JAXB annotated Java type that can be used convert
- * records into Java objects and vice versa using JAXB marshalling/unmarshalling mechanisms. Reading
- * the source will generate a {@code PCollection} of the given JAXB annotated Java type.
- * Optionally users may provide a minimum size of a bundle that should be created for the source.
- *
- * <p>The following example shows how to read from {@link XmlSource} in a Dataflow pipeline:
- *
- * <pre>
- * {@code
- * XmlSource<String> source = XmlSource.<String>from(file.toPath().toString())
- *     .withRootElement("root")
- *     .withRecordElement("record")
- *     .withRecordClass(Record.class);
- * PCollection<String> output = p.apply(Read.from(source));
- * }
- * </pre>
- *
- * <p>Currently, only XML files that use single-byte characters are supported. Using a file that
- * contains multi-byte characters may result in data loss or duplication.
- *
- * <p>To use {@link XmlSource}:
- * <ol>
- *   <li>Explicitly declare a dependency on org.codehaus.woodstox:stax2-api</li>
- *   <li>Include a compatible implementation on the classpath at run-time,
- *       such as org.codehaus.woodstox:woodstox-core-asl</li>
- * </ol>
- *
- * <p>These dependencies have been declared as optional in Maven sdk/pom.xml file of
- * Google Cloud Dataflow.
- *
- * <p><h3>Permissions</h3>
- * Permission requirements depend on the
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner PipelineRunner} that is
- * used to execute the Dataflow job. Please refer to the documentation of corresponding
- * {@link PipelineRunner PipelineRunners} for more details.
- *
- * @param <T> Type of the objects that represent the records of the XML file. The
- *        {@code PCollection} generated by this source will be of this type.
- */
-// CHECKSTYLE.ON: JavadocStyle
-public class XmlSource<T> extends FileBasedSource<T> {
-
-  private static final String XML_VERSION = "1.1";
-  private static final int DEFAULT_MIN_BUNDLE_SIZE = 8 * 1024;
-  private final String rootElement;
-  private final String recordElement;
-  private final Class<T> recordClass;
-
-  /**
-   * Creates an XmlSource for a single XML file or a set of XML files defined by a Java "glob" file
-   * pattern. Each XML file should be of the form defined in {@link XmlSource}.
-   */
-  public static <T> XmlSource<T> from(String fileOrPatternSpec) {
-    return new XmlSource<>(fileOrPatternSpec, DEFAULT_MIN_BUNDLE_SIZE, null, null, null);
-  }
-
-  /**
-   * Sets name of the root element of the XML document. This will be used to create a valid starting
-   * root element when initiating a bundle of records created from an XML document. This is a
-   * required parameter.
-   */
-  public XmlSource<T> withRootElement(String rootElement) {
-    return new XmlSource<>(
-        getFileOrPatternSpec(), getMinBundleSize(), rootElement, recordElement, recordClass);
-  }
-
-  /**
-   * Sets name of the record element of the XML document. This will be used to determine offset of
-   * the first record of a bundle created from the XML document. This is a required parameter.
-   */
-  public XmlSource<T> withRecordElement(String recordElement) {
-    return new XmlSource<>(
-        getFileOrPatternSpec(), getMinBundleSize(), rootElement, recordElement, recordClass);
-  }
-
-  /**
-   * Sets a JAXB annotated class that can be populated using a record of the provided XML file. This
-   * will be used when unmarshalling record objects from the XML file.  This is a required
-   * parameter.
-   */
-  public XmlSource<T> withRecordClass(Class<T> recordClass) {
-    return new XmlSource<>(
-        getFileOrPatternSpec(), getMinBundleSize(), rootElement, recordElement, recordClass);
-  }
-
-  /**
-   * Sets a parameter {@code minBundleSize} for the minimum bundle size of the source. Please refer
-   * to {@link OffsetBasedSource} for the definition of minBundleSize.  This is an optional
-   * parameter.
-   */
-  public XmlSource<T> withMinBundleSize(long minBundleSize) {
-    return new XmlSource<>(
-        getFileOrPatternSpec(), minBundleSize, rootElement, recordElement, recordClass);
-  }
-
-  private XmlSource(String fileOrPattern, long minBundleSize, String rootElement,
-      String recordElement, Class<T> recordClass) {
-    super(fileOrPattern, minBundleSize);
-    this.rootElement = rootElement;
-    this.recordElement = recordElement;
-    this.recordClass = recordClass;
-  }
-
-  private XmlSource(String fileOrPattern, long minBundleSize, long startOffset, long endOffset,
-      String rootElement, String recordElement, Class<T> recordClass) {
-    super(fileOrPattern, minBundleSize, startOffset, endOffset);
-    this.rootElement = rootElement;
-    this.recordElement = recordElement;
-    this.recordClass = recordClass;
-  }
-
-  @Override
-  protected FileBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) {
-    return new XmlSource<T>(
-        fileName, getMinBundleSize(), start, end, rootElement, recordElement, recordClass);
-  }
-
-  @Override
-  protected FileBasedReader<T> createSingleFileReader(PipelineOptions options) {
-    return new XMLReader<T>(this);
-  }
-
-  @Override
-  public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-    return false;
-  }
-
-  @Override
-  public void validate() {
-    super.validate();
-    Preconditions.checkNotNull(
-        rootElement, "rootElement is null. Use builder method withRootElement() to set this.");
-    Preconditions.checkNotNull(
-        recordElement,
-        "recordElement is null. Use builder method withRecordElement() to set this.");
-    Preconditions.checkNotNull(
-        recordClass, "recordClass is null. Use builder method withRecordClass() to set this.");
-  }
-
-  @Override
-  public Coder<T> getDefaultOutputCoder() {
-    return JAXBCoder.of(recordClass);
-  }
-
-  public String getRootElement() {
-    return rootElement;
-  }
-
-  public String getRecordElement() {
-    return recordElement;
-  }
-
-  public Class<T> getRecordClass() {
-    return recordClass;
-  }
-
-  /**
-   * A {@link Source.Reader} for reading JAXB annotated Java objects from an XML file. The XML
-   * file should be of the form defined at {@link XmlSource}.
-   *
-   * <p>Timestamped values are currently unsupported - all values implicitly have the timestamp
-   * of {@code BoundedWindow.TIMESTAMP_MIN_VALUE}.
-   *
-   * @param <T> Type of objects that will be read by the reader.
-   */
-  private static class XMLReader<T> extends FileBasedReader<T> {
-    // The amount of bytes read from the channel to memory when determining the starting offset of
-    // the first record in a bundle. After matching to starting offset of the first record the
-    // remaining bytes read to this buffer and the bytes still not read from the channel are used to
-    // create the XML parser.
-    private static final int BUF_SIZE = 1024;
-
-    // This should be the maximum number of bytes a character will encode to, for any encoding
-    // supported by XmlSource. Currently this is set to 4 since UTF-8 characters may be
-    // four bytes.
-    private static final int MAX_CHAR_BYTES = 4;
-
-    // In order to support reading starting in the middle of an XML file, we construct an imaginary
-    // well-formed document (a header and root tag followed by the contents of the input starting at
-    // the record boundary) and feed it to the parser. Because of this, the offset reported by the
-    // XML parser is not the same as offset in the original file. They differ by a constant amount:
-    // offsetInOriginalFile = parser.getLocation().getCharacterOffset() + parserBaseOffset;
-    // Note that this is true only for files with single-byte characters.
-    // It appears that, as of writing, there does not exist a Java XML parser capable of correctly
-    // reporting byte offsets of elements in the presence of multi-byte characters.
-    private long parserBaseOffset = 0;
-    private boolean readingStarted = false;
-
-    // If true, the current bundle does not contain any records.
-    private boolean emptyBundle = false;
-
-    private Unmarshaller jaxbUnmarshaller = null;
-    private XMLStreamReader parser = null;
-
-    private T currentRecord = null;
-
-    // Byte offset of the current record in the XML file provided when creating the source.
-    private long currentByteOffset = 0;
-
-    public XMLReader(XmlSource<T> source) {
-      super(source);
-
-      // Set up a JAXB Unmarshaller that can be used to unmarshall record objects.
-      try {
-        JAXBContext jaxbContext = JAXBContext.newInstance(getCurrentSource().recordClass);
-        jaxbUnmarshaller = jaxbContext.createUnmarshaller();
-
-        // Throw errors if validation fails. JAXB by default ignores validation errors.
-        jaxbUnmarshaller.setEventHandler(new ValidationEventHandler() {
-          @Override
-          public boolean handleEvent(ValidationEvent event) {
-            throw new RuntimeException(event.getMessage(), event.getLinkedException());
-          }
-        });
-      } catch (JAXBException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public synchronized XmlSource<T> getCurrentSource() {
-      return (XmlSource<T>) super.getCurrentSource();
-    }
-
-    @Override
-    protected void startReading(ReadableByteChannel channel) throws IOException {
-      // This method determines the correct starting offset of the first record by reading bytes
-      // from the ReadableByteChannel. This implementation does not need the channel to be a
-      // SeekableByteChannel.
-      // The method tries to determine the first record element in the byte channel. The first
-      // record must start with the characters "<recordElement" where "recordElement" is the
-      // record element of the XML document described above. For the match to be complete this
-      // has to be followed by one of following.
-      // * any whitespace character
-      // * '>' character
-      // * '/' character (to support empty records).
-      //
-      // After this match this method creates the XML parser for parsing the XML document,
-      // feeding it a fake document consisting of an XML header and the <rootElement> tag followed
-      // by the contents of channel starting from <recordElement. The <rootElement> tag may be never
-      // closed.
-
-      // This stores any bytes that should be used prior to the remaining bytes of the channel when
-      // creating an XML parser object.
-      ByteArrayOutputStream preambleByteBuffer = new ByteArrayOutputStream();
-      // A dummy declaration and root for the document with proper XML version and encoding. Without
-      // this XML parsing may fail or may produce incorrect results.
-
-      byte[] dummyStartDocumentBytes =
-          ("<?xml version=\"" + XML_VERSION + "\" encoding=\"UTF-8\" ?>"
-              + "<" + getCurrentSource().rootElement + ">").getBytes(StandardCharsets.UTF_8);
-      preambleByteBuffer.write(dummyStartDocumentBytes);
-      // Gets the byte offset (in the input file) of the first record in ReadableByteChannel. This
-      // method returns the offset and stores any bytes that should be used when creating the XML
-      // parser in preambleByteBuffer.
-      long offsetInFileOfRecordElement =
-          getFirstOccurenceOfRecordElement(channel, preambleByteBuffer);
-      if (offsetInFileOfRecordElement < 0) {
-        // Bundle has no records. So marking this bundle as an empty bundle.
-        emptyBundle = true;
-        return;
-      } else {
-        byte[] preambleBytes = preambleByteBuffer.toByteArray();
-        currentByteOffset = offsetInFileOfRecordElement;
-        setUpXMLParser(channel, preambleBytes);
-        parserBaseOffset = offsetInFileOfRecordElement - dummyStartDocumentBytes.length;
-      }
-      readingStarted = true;
-    }
-
-    // Gets the first occurrence of the next record within the given ReadableByteChannel. Puts
-    // any bytes read past the starting offset of the next record back to the preambleByteBuffer.
-    // If a record is found, returns the starting offset of the record, otherwise
-    // returns -1.
-    private long getFirstOccurenceOfRecordElement(
-        ReadableByteChannel channel, ByteArrayOutputStream preambleByteBuffer) throws IOException {
-      int byteIndexInRecordElementToMatch = 0;
-      // Index of the byte in the string "<recordElement" to be matched
-      // against the current byte from the stream.
-      boolean recordStartBytesMatched = false; // "<recordElement" matched. Still have to match the
-      // next character to confirm if this is a positive match.
-      boolean fullyMatched = false; // If true, record element was fully matched.
-
-      // This gives the offset of the byte currently being read. We do a '-1' here since we
-      // increment this value at the beginning of the while loop below.
-      long offsetInFileOfCurrentByte = getCurrentSource().getStartOffset() - 1;
-      long startingOffsetInFileOfCurrentMatch = -1;
-      // If this is non-negative, currently there is a match in progress and this value gives the
-      // starting offset of the match currently being conducted.
-      boolean matchStarted = false; // If true, a match is currently in progress.
-
-      // These two values are used to determine the character immediately following a match for
-      // "<recordElement". Please see the comment for 'MAX_CHAR_BYTES' above.
-      byte[] charBytes = new byte[MAX_CHAR_BYTES];
-      int charBytesFound = 0;
-
-      ByteBuffer buf = ByteBuffer.allocate(BUF_SIZE);
-      byte[] recordStartBytes =
-          ("<" + getCurrentSource().recordElement).getBytes(StandardCharsets.UTF_8);
-
-      outer: while (channel.read(buf) > 0) {
-        buf.flip();
-        while (buf.hasRemaining()) {
-          offsetInFileOfCurrentByte++;
-          byte b = buf.get();
-          boolean reset = false;
-          if (recordStartBytesMatched) {
-            // We already matched "<recordElement" reading the next character to determine if this
-            // is a positive match for a new record.
-            charBytes[charBytesFound] = b;
-            charBytesFound++;
-            Character c = null;
-            if (charBytesFound == charBytes.length) {
-              CharBuffer charBuf = CharBuffer.allocate(1);
-              InputStream charBufStream = new ByteArrayInputStream(charBytes);
-              java.io.Reader reader =
-                  new InputStreamReader(charBufStream, StandardCharsets.UTF_8);
-              int read = reader.read();
-              if (read <= 0) {
-                return -1;
-              }
-              charBuf.flip();
-              c = (char) read;
-            } else {
-              continue;
-            }
-
-            // Record start may be of following forms
-            // * "<recordElement<whitespace>..."
-            // * "<recordElement>..."
-            // * "<recordElement/..."
-            if (Character.isWhitespace(c) || c == '>' || c == '/') {
-              fullyMatched = true;
-              // Add the recordStartBytes and charBytes to preambleByteBuffer since these were
-              // already read from the channel.
-              preambleByteBuffer.write(recordStartBytes);
-              preambleByteBuffer.write(charBytes);
-              // Also add the rest of the current buffer to preambleByteBuffer.
-              while (buf.hasRemaining()) {
-                preambleByteBuffer.write(buf.get());
-              }
-              break outer;
-            } else {
-              // Matching was unsuccessful. Reset the buffer to include bytes read for the char.
-              ByteBuffer newbuf = ByteBuffer.allocate(BUF_SIZE);
-              newbuf.put(charBytes);
-              offsetInFileOfCurrentByte -= charBytes.length;
-              while (buf.hasRemaining()) {
-                newbuf.put(buf.get());
-              }
-              newbuf.flip();
-              buf = newbuf;
-
-              // Ignore everything and try again starting from the current buffer.
-              reset = true;
-            }
-          } else if (b == recordStartBytes[byteIndexInRecordElementToMatch]) {
-            // Next byte matched.
-            if (!matchStarted) {
-              // Match was for the first byte, record the starting offset.
-              matchStarted = true;
-              startingOffsetInFileOfCurrentMatch = offsetInFileOfCurrentByte;
-            }
-            byteIndexInRecordElementToMatch++;
-          } else {
-            // Not a match. Ignore everything and try again starting at current point.
-            reset = true;
-          }
-          if (reset) {
-            // Clear variables and try to match starting from the next byte.
-            byteIndexInRecordElementToMatch = 0;
-            startingOffsetInFileOfCurrentMatch = -1;
-            matchStarted = false;
-            recordStartBytesMatched = false;
-            charBytes = new byte[MAX_CHAR_BYTES];
-            charBytesFound = 0;
-          }
-          if (byteIndexInRecordElementToMatch == recordStartBytes.length) {
-            // "<recordElement" matched. Need to still check next byte since this might be an
-            // element that has "recordElement" as a prefix.
-            recordStartBytesMatched = true;
-          }
-        }
-        buf.clear();
-      }
-
-      if (!fullyMatched) {
-        return -1;
-      } else {
-        return startingOffsetInFileOfCurrentMatch;
-      }
-    }
-
-    private void setUpXMLParser(ReadableByteChannel channel, byte[] lookAhead) throws IOException {
-      try {
-        // We use Woodstox because the StAX implementation provided by OpenJDK reports
-        // character locations incorrectly. Note that Woodstox still currently reports *byte*
-        // locations incorrectly when parsing documents that contain multi-byte characters.
-        XMLInputFactory2 xmlInputFactory = (XMLInputFactory2) XMLInputFactory.newInstance();
-        this.parser = xmlInputFactory.createXMLStreamReader(
-            new SequenceInputStream(
-                new ByteArrayInputStream(lookAhead), Channels.newInputStream(channel)),
-            "UTF-8");
-
-        // Current offset should be the offset before reading the record element.
-        while (true) {
-          int event = parser.next();
-          if (event == XMLStreamConstants.START_ELEMENT) {
-            String localName = parser.getLocalName();
-            if (localName.equals(getCurrentSource().recordElement)) {
-              break;
-            }
-          }
-        }
-      } catch (FactoryConfigurationError | XMLStreamException e) {
-        throw new IOException(e);
-      }
-    }
-
-    @Override
-    protected boolean readNextRecord() throws IOException {
-      if (emptyBundle) {
-        currentByteOffset = Long.MAX_VALUE;
-        return false;
-      }
-      try {
-        // Update current offset and check if the next value is the record element.
-        currentByteOffset = parserBaseOffset + parser.getLocation().getCharacterOffset();
-        while (parser.getEventType() != XMLStreamConstants.START_ELEMENT) {
-          parser.next();
-          currentByteOffset = parserBaseOffset + parser.getLocation().getCharacterOffset();
-          if (parser.getEventType() == XMLStreamConstants.END_DOCUMENT) {
-            currentByteOffset = Long.MAX_VALUE;
-            return false;
-          }
-        }
-        JAXBElement<T> jb = jaxbUnmarshaller.unmarshal(parser, getCurrentSource().recordClass);
-        currentRecord = jb.getValue();
-        return true;
-      } catch (JAXBException | XMLStreamException e) {
-        throw new IOException(e);
-      }
-    }
-
-    @Override
-    public T getCurrent() throws NoSuchElementException {
-      if (!readingStarted) {
-        throw new NoSuchElementException();
-      }
-      return currentRecord;
-    }
-
-    @Override
-    protected boolean isAtSplitPoint() {
-      // Every record is at a split point.
-      return true;
-    }
-
-    @Override
-    protected long getCurrentOffset() {
-      return currentByteOffset;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java
deleted file mode 100644
index 0ce2a5e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java
+++ /dev/null
@@ -1,989 +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 com.google.cloud.dataflow.sdk.io.bigtable;
-
-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;
-
-import com.google.bigtable.v1.Mutation;
-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.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Proto2Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.io.range.ByteKey;
-import com.google.cloud.dataflow.sdk.io.range.ByteKeyRange;
-import com.google.cloud.dataflow.sdk.io.range.ByteKeyRangeTracker;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Empty;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import javax.annotation.Nullable;
-
-/**
- * A bounded source and sink for Google Cloud Bigtable.
- *
- * <p>For more information, see the online documentation at
- * <a href="https://cloud.google.com/bigtable/">Google Cloud Bigtable</a>.
- *
- * <h3>Reading from Cloud Bigtable</h3>
- *
- * <p>The Bigtable source returns a set of rows from a single table, returning a
- * {@code PCollection<Row>}.
- *
- * <p>To configure a Cloud Bigtable source, you must supply a table id and a {@link BigtableOptions}
- * or builder configured with the project and other information necessary to identify the
- * Bigtable cluster. A {@link RowFilter} may also optionally be specified using
- * {@link BigtableIO.Read#withRowFilter}. For example:
- *
- * <pre>{@code
- * BigtableOptions.Builder optionsBuilder =
- *     new BigtableOptions.Builder()
- *         .setProjectId("project")
- *         .setClusterId("cluster")
- *         .setZoneId("zone");
- *
- * Pipeline p = ...;
- *
- * // Scan the entire table.
- * p.apply("read",
- *     BigtableIO.read()
- *         .withBigtableOptions(optionsBuilder)
- *         .withTableId("table"));
- *
- * // Scan a subset of rows that match the specified row filter.
- * p.apply("filtered read",
- *     BigtableIO.read()
- *         .withBigtableOptions(optionsBuilder)
- *         .withTableId("table")
- *         .withRowFilter(filter));
- * }</pre>
- *
- * <h3>Writing to Cloud Bigtable</h3>
- *
- * <p>The Bigtable sink executes a set of row mutations on a single table. It takes as input a
- * {@link PCollection PCollection&lt;KV&lt;ByteString, Iterable&lt;Mutation&gt;&gt;&gt;}, where the
- * {@link ByteString} is the key of the row being mutated, and each {@link Mutation} represents an
- * idempotent transformation to that row.
- *
- * <p>To configure a Cloud Bigtable sink, you must supply a table id and a {@link BigtableOptions}
- * or builder configured with the project and other information necessary to identify the
- * Bigtable cluster, for example:
- *
- * <pre>{@code
- * BigtableOptions.Builder optionsBuilder =
- *     new BigtableOptions.Builder()
- *         .setProjectId("project")
- *         .setClusterId("cluster")
- *         .setZoneId("zone");
- *
- * PCollection<KV<ByteString, Iterable<Mutation>>> data = ...;
- *
- * data.apply("write",
- *     BigtableIO.write()
- *         .withBigtableOptions(optionsBuilder)
- *         .withTableId("table"));
- * }</pre>
- *
- * <h3>Experimental</h3>
- *
- * <p>This connector for Cloud Bigtable is considered experimental and may break or receive
- * backwards-incompatible changes in future versions of the Cloud Dataflow SDK. Cloud Bigtable is
- * in Beta, and thus it may introduce breaking changes in future revisions of its service or APIs.
- *
- * <h3>Permissions</h3>
- *
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding
- * {@link PipelineRunner PipelineRunners} for more details.
- */
-@Experimental
-public class BigtableIO {
-  private static final Logger logger = LoggerFactory.getLogger(BigtableIO.class);
-
-  /**
-   * Creates an uninitialized {@link BigtableIO.Read}. Before use, the {@code Read} must be
-   * initialized with a
-   * {@link BigtableIO.Read#withBigtableOptions(BigtableOptions) BigtableOptions} that specifies
-   * the source Cloud Bigtable cluster, and a {@link BigtableIO.Read#withTableId tableId} that
-   * specifies which table to read. A {@link RowFilter} may also optionally be specified using
-   * {@link BigtableIO.Read#withRowFilter}.
-   */
-  @Experimental
-  public static Read read() {
-    return new Read(null, "", null, null);
-  }
-
-  /**
-   * Creates an uninitialized {@link BigtableIO.Write}. Before use, the {@code Write} must be
-   * initialized with a
-   * {@link BigtableIO.Write#withBigtableOptions(BigtableOptions) BigtableOptions} that specifies
-   * the destination Cloud Bigtable cluster, and a {@link BigtableIO.Write#withTableId tableId} that
-   * specifies which table to write.
-   */
-  @Experimental
-  public static Write write() {
-    return new Write(null, "", null);
-  }
-
-  /**
-   * A {@link PTransform} that reads from Google Cloud Bigtable. See the class-level Javadoc on
-   * {@link BigtableIO} for more information.
-   *
-   * @see BigtableIO
-   */
-  @Experimental
-  public static class Read extends PTransform<PBegin, PCollection<Row>> {
-    /**
-     * Returns a new {@link BigtableIO.Read} that will read from the Cloud Bigtable cluster
-     * indicated by the given options, and using any other specified customizations.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read withBigtableOptions(BigtableOptions options) {
-      checkNotNull(options, "options");
-      return withBigtableOptions(options.toBuilder());
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Read} that will read from the Cloud Bigtable cluster
-     * indicated by the given options, and using any other specified customizations.
-     *
-     * <p>Clones the given {@link BigtableOptions} builder so that any further changes
-     * will have no effect on the returned {@link BigtableIO.Read}.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) {
-      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();
-      BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getUserAgent()).build();
-      return new Read(optionsWithAgent, tableId, filter, bigtableService);
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Read} that will filter the rows read from Cloud Bigtable
-     * using the given row filter.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read withRowFilter(RowFilter filter) {
-      checkNotNull(filter, "filter");
-      return new Read(options, tableId, filter, bigtableService);
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Read} that will read from the specified table.
-     *
-     * <p>Does not modify this object.
-     */
-    public Read withTableId(String tableId) {
-      checkNotNull(tableId, "tableId");
-      return new Read(options, tableId, filter, bigtableService);
-    }
-
-    /**
-     * Returns the Google Cloud Bigtable cluster being read from, and other parameters.
-     */
-    public BigtableOptions getBigtableOptions() {
-      return options;
-    }
-
-    /**
-     * Returns the table being read from.
-     */
-    public String getTableId() {
-      return tableId;
-    }
-
-    @Override
-    public PCollection<Row> apply(PBegin input) {
-      BigtableSource source =
-          new BigtableSource(getBigtableService(), tableId, filter, ByteKeyRange.ALL_KEYS, null);
-      return input.getPipeline().apply(com.google.cloud.dataflow.sdk.io.Read.from(source));
-    }
-
-    @Override
-    public void validate(PBegin input) {
-      checkArgument(options != null, "BigtableOptions not specified");
-      checkArgument(!tableId.isEmpty(), "Table ID not specified");
-      try {
-        checkArgument(
-            getBigtableService().tableExists(tableId), "Table %s does not exist", tableId);
-      } catch (IOException e) {
-        logger.warn("Error checking whether table {} exists; proceeding.", tableId, e);
-      }
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(Read.class)
-          .add("options", options)
-          .add("tableId", tableId)
-          .add("filter", filter)
-          .toString();
-    }
-
-    /////////////////////////////////////////////////////////////////////////////////////////
-    /**
-     * Used to define the Cloud Bigtable cluster and any options for the networking layer.
-     * Cannot actually be {@code null} at validation time, but may start out {@code null} while
-     * source is being built.
-     */
-    @Nullable private final BigtableOptions options;
-    private final String tableId;
-    @Nullable private final RowFilter filter;
-    @Nullable private final BigtableService bigtableService;
-
-    private Read(
-        @Nullable BigtableOptions options,
-        String tableId,
-        @Nullable RowFilter filter,
-        @Nullable BigtableService bigtableService) {
-      this.options = options;
-      this.tableId = checkNotNull(tableId, "tableId");
-      this.filter = filter;
-      this.bigtableService = bigtableService;
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Read} that will read using the given Cloud Bigtable
-     * service implementation.
-     *
-     * <p>This is used for testing.
-     *
-     * <p>Does not modify this object.
-     */
-    Read withBigtableService(BigtableService bigtableService) {
-      checkNotNull(bigtableService, "bigtableService");
-      return new Read(options, tableId, filter, bigtableService);
-    }
-
-    /**
-     * Helper function that either returns the mock Bigtable service supplied by
-     * {@link #withBigtableService} or creates and returns an implementation that talks to
-     * {@code Cloud Bigtable}.
-     */
-    private BigtableService getBigtableService() {
-      if (bigtableService != null) {
-        return bigtableService;
-      }
-      return new BigtableServiceImpl(options);
-    }
-  }
-
-  /**
-   * A {@link PTransform} that writes to Google Cloud Bigtable. See the class-level Javadoc on
-   * {@link BigtableIO} for more information.
-   *
-   * @see BigtableIO
-   */
-  @Experimental
-  public static class Write
-      extends PTransform<PCollection<KV<ByteString, Iterable<Mutation>>>, PDone> {
-    /**
-     * Used to define the Cloud Bigtable cluster and any options for the networking layer.
-     * Cannot actually be {@code null} at validation time, but may start out {@code null} while
-     * source is being built.
-     */
-    @Nullable private final BigtableOptions options;
-    private final String tableId;
-    @Nullable private final BigtableService bigtableService;
-
-    private Write(
-        @Nullable BigtableOptions options,
-        String tableId,
-        @Nullable BigtableService bigtableService) {
-      this.options = options;
-      this.tableId = checkNotNull(tableId, "tableId");
-      this.bigtableService = bigtableService;
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Write} that will write to the Cloud Bigtable cluster
-     * indicated by the given options, and using any other specified customizations.
-     *
-     * <p>Does not modify this object.
-     */
-    public Write withBigtableOptions(BigtableOptions options) {
-      checkNotNull(options, "options");
-      return withBigtableOptions(options.toBuilder());
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Write} that will write to the Cloud Bigtable cluster
-     * indicated by the given options, and using any other specified customizations.
-     *
-     * <p>Clones the given {@link BigtableOptions} builder so that any further changes
-     * will have no effect on the returned {@link BigtableIO.Write}.
-     *
-     * <p>Does not modify this object.
-     */
-    public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) {
-      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();
-      BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getUserAgent()).build();
-      return new Write(optionsWithAgent, tableId, bigtableService);
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Write} that will write to the specified table.
-     *
-     * <p>Does not modify this object.
-     */
-    public Write withTableId(String tableId) {
-      checkNotNull(tableId, "tableId");
-      return new Write(options, tableId, bigtableService);
-    }
-
-    /**
-     * Returns the Google Cloud Bigtable cluster being written to, and other parameters.
-     */
-    public BigtableOptions getBigtableOptions() {
-      return options;
-    }
-
-    /**
-     * Returns the table being written to.
-     */
-    public String getTableId() {
-      return tableId;
-    }
-
-    @Override
-    public PDone apply(PCollection<KV<ByteString, Iterable<Mutation>>> input) {
-      Sink sink = new Sink(tableId, getBigtableService());
-      return input.apply(com.google.cloud.dataflow.sdk.io.Write.to(sink));
-    }
-
-    @Override
-    public void validate(PCollection<KV<ByteString, Iterable<Mutation>>> input) {
-      checkArgument(options != null, "BigtableOptions not specified");
-      checkArgument(!tableId.isEmpty(), "Table ID not specified");
-      try {
-        checkArgument(
-            getBigtableService().tableExists(tableId), "Table %s does not exist", tableId);
-      } catch (IOException e) {
-        logger.warn("Error checking whether table {} exists; proceeding.", tableId, e);
-      }
-    }
-
-    /**
-     * Returns a new {@link BigtableIO.Write} that will write using the given Cloud Bigtable
-     * service implementation.
-     *
-     * <p>This is used for testing.
-     *
-     * <p>Does not modify this object.
-     */
-    Write withBigtableService(BigtableService bigtableService) {
-      checkNotNull(bigtableService, "bigtableService");
-      return new Write(options, tableId, bigtableService);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(Write.class)
-          .add("options", options)
-          .add("tableId", tableId)
-          .toString();
-    }
-
-    /**
-     * Helper function that either returns the mock Bigtable service supplied by
-     * {@link #withBigtableService} or creates and returns an implementation that talks to
-     * {@code Cloud Bigtable}.
-     */
-    private BigtableService getBigtableService() {
-      if (bigtableService != null) {
-        return bigtableService;
-      }
-      return new BigtableServiceImpl(options);
-    }
-  }
-
-  //////////////////////////////////////////////////////////////////////////////////////////
-  /** Disallow construction of utility class. */
-  private BigtableIO() {}
-
-  static class BigtableSource extends BoundedSource<Row> {
-    public BigtableSource(
-        BigtableService service,
-        String tableId,
-        @Nullable RowFilter filter,
-        ByteKeyRange range,
-        Long estimatedSizeBytes) {
-      this.service = service;
-      this.tableId = tableId;
-      this.filter = filter;
-      this.range = range;
-      this.estimatedSizeBytes = estimatedSizeBytes;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(BigtableSource.class)
-          .add("tableId", tableId)
-          .add("filter", filter)
-          .add("range", range)
-          .add("estimatedSizeBytes", estimatedSizeBytes)
-          .toString();
-    }
-
-    ////// Private state and internal implementation details //////
-    private final BigtableService service;
-    @Nullable private final String tableId;
-    @Nullable private final RowFilter filter;
-    private final ByteKeyRange range;
-    @Nullable private Long estimatedSizeBytes;
-    @Nullable private transient List<SampleRowKeysResponse> sampleRowKeys;
-
-    protected BigtableSource withStartKey(ByteKey startKey) {
-      checkNotNull(startKey, "startKey");
-      return new BigtableSource(
-          service, tableId, filter, range.withStartKey(startKey), estimatedSizeBytes);
-    }
-
-    protected BigtableSource withEndKey(ByteKey endKey) {
-      checkNotNull(endKey, "endKey");
-      return new BigtableSource(
-          service, tableId, filter, range.withEndKey(endKey), estimatedSizeBytes);
-    }
-
-    protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) {
-      checkNotNull(estimatedSizeBytes, "estimatedSizeBytes");
-      return new BigtableSource(service, tableId, filter, range, estimatedSizeBytes);
-    }
-
-    /**
-     * Makes an API call to the Cloud Bigtable service that gives information about tablet key
-     * boundaries and estimated sizes. We can use these samples to ensure that splits are on
-     * different tablets, and possibly generate sub-splits within tablets.
-     */
-    private List<SampleRowKeysResponse> getSampleRowKeys() throws IOException {
-      return service.getSampleRowKeys(this);
-    }
-
-    @Override
-    public List<BigtableSource> splitIntoBundles(
-        long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
-      // Update the desiredBundleSizeBytes in order to limit the
-      // number of splits to maximumNumberOfSplits.
-      long maximumNumberOfSplits = 4000;
-      long sizeEstimate = getEstimatedSizeBytes(options);
-      desiredBundleSizeBytes =
-          Math.max(sizeEstimate / maximumNumberOfSplits, desiredBundleSizeBytes);
-
-      // Delegate to testable helper.
-      return splitIntoBundlesBasedOnSamples(desiredBundleSizeBytes, getSampleRowKeys());
-    }
-
-    /** Helper that splits this source into bundles based on Cloud Bigtable sampled row keys. */
-    private List<BigtableSource> splitIntoBundlesBasedOnSamples(
-        long desiredBundleSizeBytes, List<SampleRowKeysResponse> sampleRowKeys) {
-      // There are no regions, or no samples available. Just scan the entire range.
-      if (sampleRowKeys.isEmpty()) {
-        logger.info("Not splitting source {} because no sample row keys are available.", this);
-        return Collections.singletonList(this);
-      }
-
-      logger.info(
-          "About to split into bundles of size {} with sampleRowKeys length {} first element {}",
-          desiredBundleSizeBytes,
-          sampleRowKeys.size(),
-          sampleRowKeys.get(0));
-
-      // Loop through all sampled responses and generate splits from the ones that overlap the
-      // scan range. The main complication is that we must track the end range of the previous
-      // sample to generate good ranges.
-      ByteKey lastEndKey = ByteKey.EMPTY;
-      long lastOffset = 0;
-      ImmutableList.Builder<BigtableSource> splits = ImmutableList.builder();
-      for (SampleRowKeysResponse response : sampleRowKeys) {
-        ByteKey responseEndKey = ByteKey.of(response.getRowKey());
-        long responseOffset = response.getOffsetBytes();
-        checkState(
-            responseOffset >= lastOffset,
-            "Expected response byte offset %s to come after the last offset %s",
-            responseOffset,
-            lastOffset);
-
-        if (!range.overlaps(ByteKeyRange.of(lastEndKey, responseEndKey))) {
-          // This region does not overlap the scan, so skip it.
-          lastOffset = responseOffset;
-          lastEndKey = responseEndKey;
-          continue;
-        }
-
-        // Calculate the beginning of the split as the larger of startKey and the end of the last
-        // split. Unspecified start is smallest key so is correctly treated as earliest key.
-        ByteKey splitStartKey = lastEndKey;
-        if (splitStartKey.compareTo(range.getStartKey()) < 0) {
-          splitStartKey = range.getStartKey();
-        }
-
-        // Calculate the end of the split as the smaller of endKey and the end of this sample. Note
-        // that range.containsKey handles the case when range.getEndKey() is empty.
-        ByteKey splitEndKey = responseEndKey;
-        if (!range.containsKey(splitEndKey)) {
-          splitEndKey = range.getEndKey();
-        }
-
-        // We know this region overlaps the desired key range, and we know a rough estimate of its
-        // size. Split the key range into bundle-sized chunks and then add them all as splits.
-        long sampleSizeBytes = responseOffset - lastOffset;
-        List<BigtableSource> subSplits =
-            splitKeyRangeIntoBundleSizedSubranges(
-                sampleSizeBytes,
-                desiredBundleSizeBytes,
-                ByteKeyRange.of(splitStartKey, splitEndKey));
-        splits.addAll(subSplits);
-
-        // Move to the next region.
-        lastEndKey = responseEndKey;
-        lastOffset = responseOffset;
-      }
-
-      // We must add one more region after the end of the samples if both these conditions hold:
-      //  1. we did not scan to the end yet (lastEndKey is concrete, not 0-length).
-      //  2. we want to scan to the end (endKey is empty) or farther (lastEndKey < endKey).
-      if (!lastEndKey.isEmpty()
-          && (range.getEndKey().isEmpty() || lastEndKey.compareTo(range.getEndKey()) < 0)) {
-        splits.add(this.withStartKey(lastEndKey).withEndKey(range.getEndKey()));
-      }
-
-      List<BigtableSource> ret = splits.build();
-      logger.info("Generated {} splits. First split: {}", ret.size(), ret.get(0));
-      return ret;
-    }
-
-    @Override
-    public long getEstimatedSizeBytes(PipelineOptions options) throws IOException {
-      // Delegate to testable helper.
-      if (estimatedSizeBytes == null) {
-        estimatedSizeBytes = getEstimatedSizeBytesBasedOnSamples(getSampleRowKeys());
-      }
-      return estimatedSizeBytes;
-    }
-
-    /**
-     * Computes the estimated size in bytes based on the total size of all samples that overlap
-     * the key range this source will scan.
-     */
-    private long getEstimatedSizeBytesBasedOnSamples(List<SampleRowKeysResponse> samples) {
-      long estimatedSizeBytes = 0;
-      long lastOffset = 0;
-      ByteKey currentStartKey = ByteKey.EMPTY;
-      // Compute the total estimated size as the size of each sample that overlaps the scan range.
-      // TODO: In future, Bigtable service may provide finer grained APIs, e.g., to sample given a
-      // filter or to sample on a given key range.
-      for (SampleRowKeysResponse response : samples) {
-        ByteKey currentEndKey = ByteKey.of(response.getRowKey());
-        long currentOffset = response.getOffsetBytes();
-        if (!currentStartKey.isEmpty() && currentStartKey.equals(currentEndKey)) {
-          // Skip an empty region.
-          lastOffset = currentOffset;
-          continue;
-        } else if (range.overlaps(ByteKeyRange.of(currentStartKey, currentEndKey))) {
-          estimatedSizeBytes += currentOffset - lastOffset;
-        }
-        currentStartKey = currentEndKey;
-        lastOffset = currentOffset;
-      }
-      return estimatedSizeBytes;
-    }
-
-    /**
-     * Cloud Bigtable returns query results ordered by key.
-     */
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-      return true;
-    }
-
-    @Override
-    public BoundedReader<Row> createReader(PipelineOptions options) throws IOException {
-      return new BigtableReader(this, service);
-    }
-
-    @Override
-    public void validate() {
-      checkArgument(!tableId.isEmpty(), "tableId cannot be empty");
-    }
-
-    @Override
-    public Coder<Row> getDefaultOutputCoder() {
-      return Proto2Coder.of(Row.class);
-    }
-
-    /** Helper that splits the specified range in this source into bundles. */
-    private List<BigtableSource> splitKeyRangeIntoBundleSizedSubranges(
-        long sampleSizeBytes, long desiredBundleSizeBytes, ByteKeyRange range) {
-      // Catch the trivial cases. Split is small enough already, or this is the last region.
-      logger.debug(
-          "Subsplit for sampleSizeBytes {} and desiredBundleSizeBytes {}",
-          sampleSizeBytes,
-          desiredBundleSizeBytes);
-      if (sampleSizeBytes <= desiredBundleSizeBytes) {
-        return Collections.singletonList(
-            this.withStartKey(range.getStartKey()).withEndKey(range.getEndKey()));
-      }
-
-      checkArgument(
-          sampleSizeBytes > 0, "Sample size %s bytes must be greater than 0.", sampleSizeBytes);
-      checkArgument(
-          desiredBundleSizeBytes > 0,
-          "Desired bundle size %s bytes must be greater than 0.",
-          desiredBundleSizeBytes);
-
-      int splitCount = (int) Math.ceil(((double) sampleSizeBytes) / (desiredBundleSizeBytes));
-      List<ByteKey> splitKeys = range.split(splitCount);
-      ImmutableList.Builder<BigtableSource> splits = ImmutableList.builder();
-      Iterator<ByteKey> keys = splitKeys.iterator();
-      ByteKey prev = keys.next();
-      while (keys.hasNext()) {
-        ByteKey next = keys.next();
-        splits.add(
-            this
-                .withStartKey(prev)
-                .withEndKey(next)
-                .withEstimatedSizeBytes(sampleSizeBytes / splitCount));
-        prev = next;
-      }
-      return splits.build();
-    }
-
-    public ByteKeyRange getRange() {
-      return range;
-    }
-
-    public RowFilter getRowFilter() {
-      return filter;
-    }
-
-    public String getTableId() {
-      return tableId;
-    }
-  }
-
-  private static class BigtableReader extends BoundedReader<Row> {
-    // Thread-safety: source is protected via synchronization and is only accessed or modified
-    // inside a synchronized block (or constructor, which is the same).
-    private BigtableSource source;
-    private BigtableService service;
-    private BigtableService.Reader reader;
-    private final ByteKeyRangeTracker rangeTracker;
-    private long recordsReturned;
-
-    public BigtableReader(BigtableSource source, BigtableService service) {
-      this.source = source;
-      this.service = service;
-      rangeTracker = ByteKeyRangeTracker.of(source.getRange());
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      reader = service.createReader(getCurrentSource());
-      boolean hasRecord =
-          reader.start()
-              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()));
-      if (hasRecord) {
-        ++recordsReturned;
-      }
-      return hasRecord;
-    }
-
-    @Override
-    public synchronized BigtableSource getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      boolean hasRecord =
-          reader.advance()
-              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()));
-      if (hasRecord) {
-        ++recordsReturned;
-      }
-      return hasRecord;
-    }
-
-    @Override
-    public Row getCurrent() throws NoSuchElementException {
-      return reader.getCurrentRow();
-    }
-
-    @Override
-    public void close() throws IOException {
-      logger.info("Closing reader after reading {} records.", recordsReturned);
-      if (reader != null) {
-        reader.close();
-        reader = null;
-      }
-    }
-
-    @Override
-    public final Double getFractionConsumed() {
-      return rangeTracker.getFractionConsumed();
-    }
-
-    @Override
-    public final synchronized BigtableSource splitAtFraction(double fraction) {
-      ByteKey splitKey;
-      try {
-        splitKey = source.getRange().interpolateKey(fraction);
-      } catch (IllegalArgumentException e) {
-        logger.info("%s: Failed to interpolate key for fraction %s.", source.getRange(), fraction);
-        return null;
-      }
-      logger.debug(
-          "Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey);
-      if (!rangeTracker.trySplitAtPosition(splitKey)) {
-        return null;
-      }
-      BigtableSource primary = source.withEndKey(splitKey);
-      BigtableSource residual = source.withStartKey(splitKey);
-      this.source = primary;
-      return residual;
-    }
-  }
-
-  private static class Sink
-      extends com.google.cloud.dataflow.sdk.io.Sink<KV<ByteString, Iterable<Mutation>>> {
-
-    public Sink(String tableId, BigtableService bigtableService) {
-      this.tableId = checkNotNull(tableId, "tableId");
-      this.bigtableService = checkNotNull(bigtableService, "bigtableService");
-    }
-
-    public String getTableId() {
-      return tableId;
-    }
-
-    public BigtableService getBigtableService() {
-      return bigtableService;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(Sink.class)
-          .add("bigtableService", bigtableService)
-          .add("tableId", tableId)
-          .toString();
-    }
-
-    ///////////////////////////////////////////////////////////////////////////////
-    private final String tableId;
-    private final BigtableService bigtableService;
-
-    @Override
-    public WriteOperation<KV<ByteString, Iterable<Mutation>>, Long> createWriteOperation(
-        PipelineOptions options) {
-      return new BigtableWriteOperation(this);
-    }
-
-    /** Does nothing, as it is redundant with {@link Write#validate}. */
-    @Override
-    public void validate(PipelineOptions options) {}
-  }
-
-  private static class BigtableWriteOperation
-      extends WriteOperation<KV<ByteString, Iterable<Mutation>>, Long> {
-    private final Sink sink;
-
-    public BigtableWriteOperation(Sink sink) {
-      this.sink = sink;
-    }
-
-    @Override
-    public Writer<KV<ByteString, Iterable<Mutation>>, Long> createWriter(PipelineOptions options)
-        throws Exception {
-      return new BigtableWriter(this);
-    }
-
-    @Override
-    public void initialize(PipelineOptions options) {}
-
-    @Override
-    public void finalize(Iterable<Long> writerResults, PipelineOptions options) {
-      long count = 0;
-      for (Long value : writerResults) {
-        value += count;
-      }
-      logger.debug("Wrote {} elements to BigtableIO.Sink {}", sink);
-    }
-
-    @Override
-    public Sink getSink() {
-      return sink;
-    }
-
-    @Override
-    public Coder<Long> getWriterResultCoder() {
-      return VarLongCoder.of();
-    }
-  }
-
-  private static class BigtableWriter extends Writer<KV<ByteString, Iterable<Mutation>>, Long> {
-    private final BigtableWriteOperation writeOperation;
-    private final Sink sink;
-    private BigtableService.Writer bigtableWriter;
-    private long recordsWritten;
-    private final ConcurrentLinkedQueue<BigtableWriteException> failures;
-
-    public BigtableWriter(BigtableWriteOperation writeOperation) {
-      this.writeOperation = writeOperation;
-      this.sink = writeOperation.getSink();
-      this.failures = new ConcurrentLinkedQueue<>();
-    }
-
-    @Override
-    public void open(String uId) throws Exception {
-      bigtableWriter = sink.getBigtableService().openForWriting(sink.getTableId());
-      recordsWritten = 0;
-    }
-
-    /**
-     * If any write has asynchronously failed, fail the bundle with a useful error.
-     */
-    private void checkForFailures() throws IOException {
-      // Note that this function is never called by multiple threads and is the only place that
-      // we remove from failures, so this code is safe.
-      if (failures.isEmpty()) {
-        return;
-      }
-
-      StringBuilder logEntry = new StringBuilder();
-      int i = 0;
-      for (; i < 10 && !failures.isEmpty(); ++i) {
-        BigtableWriteException exc = failures.remove();
-        logEntry.append("\n").append(exc.getMessage());
-        if (exc.getCause() != null) {
-          logEntry.append(": ").append(exc.getCause().getMessage());
-        }
-      }
-      String message =
-          String.format(
-              "At least %d errors occurred writing to Bigtable. First %d errors: %s",
-              i + failures.size(),
-              i,
-              logEntry.toString());
-      logger.error(message);
-      throw new IOException(message);
-    }
-
-    @Override
-    public void write(KV<ByteString, Iterable<Mutation>> rowMutations) throws Exception {
-      checkForFailures();
-      Futures.addCallback(
-          bigtableWriter.writeRecord(rowMutations), new WriteExceptionCallback(rowMutations));
-      ++recordsWritten;
-    }
-
-    @Override
-    public Long close() throws Exception {
-      bigtableWriter.close();
-      bigtableWriter = null;
-      checkForFailures();
-      logger.info("Wrote {} records", recordsWritten);
-      return recordsWritten;
-    }
-
-    @Override
-    public WriteOperation<KV<ByteString, Iterable<Mutation>>, Long> getWriteOperation() {
-      return writeOperation;
-    }
-
-    private class WriteExceptionCallback implements FutureCallback<Empty> {
-      private final KV<ByteString, Iterable<Mutation>> value;
-
-      public WriteExceptionCallback(KV<ByteString, Iterable<Mutation>> value) {
-        this.value = value;
-      }
-
-      @Override
-      public void onFailure(Throwable cause) {
-        failures.add(new BigtableWriteException(value, cause));
-      }
-
-      @Override
-      public void onSuccess(Empty produced) {}
-    }
-  }
-
-  /**
-   * An exception that puts information about the failed record being written in its message.
-   */
-  static class BigtableWriteException extends IOException {
-    public BigtableWriteException(KV<ByteString, Iterable<Mutation>> record, Throwable cause) {
-      super(
-          String.format(
-              "Error mutating row %s with mutations %s",
-              record.getKey().toStringUtf8(),
-              record.getValue()),
-          cause);
-    }
-  }
-
-  /**
-   * A helper function to produce a Cloud Bigtable user agent string.
-   */
-  private static String getUserAgent() {
-    String javaVersion = System.getProperty("java.specification.version");
-    DataflowReleaseInfo info = DataflowReleaseInfo.getReleaseInfo();
-    return String.format(
-        "%s/%s (%s); %s",
-        info.getName(),
-        info.getVersion(),
-        javaVersion,
-        "0.2.3" /* TODO get Bigtable client version directly from jar. */);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableService.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableService.java
deleted file mode 100644
index 0c47f65..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableService.java
+++ /dev/null
@@ -1,110 +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 com.google.cloud.dataflow.sdk.io.bigtable;
-
-import com.google.bigtable.v1.Mutation;
-import com.google.bigtable.v1.Row;
-import com.google.bigtable.v1.SampleRowKeysResponse;
-import com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO.BigtableSource;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Empty;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * An interface for real or fake implementations of Cloud Bigtable.
- */
-interface BigtableService extends Serializable {
-
-  /**
-   * The interface of a class that can write to Cloud Bigtable.
-   */
-  interface Writer {
-    /**
-     * Writes a single row transaction to Cloud Bigtable. The key of the {@code record} is the
-     * row key to be mutated and the iterable of mutations represent the changes to be made to the
-     * row.
-     *
-     * @throws IOException if there is an error submitting the write.
-     */
-    ListenableFuture<Empty> writeRecord(KV<ByteString, Iterable<Mutation>> record)
-        throws IOException;
-
-    /**
-     * Closes the writer.
-     *
-     * @throws IOException if any writes did not succeed
-     */
-    void close() throws IOException;
-  }
-
-  /**
-   * The interface of a class that reads from Cloud Bigtable.
-   */
-  interface Reader {
-    /**
-     * Reads the first element (including initialization, such as opening a network connection) and
-     * returns true if an element was found.
-     */
-    boolean start() throws IOException;
-
-    /**
-     * Attempts to read the next element, and returns true if an element has been read.
-     */
-    boolean advance() throws IOException;
-
-    /**
-     * Closes the reader.
-     *
-     * @throws IOException if there is an error.
-     */
-    void close() throws IOException;
-
-    /**
-     * Returns the last row read by a successful start() or advance(), or throws if there is no
-     * current row because the last such call was unsuccessful.
-     */
-    Row getCurrentRow() throws NoSuchElementException;
-  }
-
-  /**
-   * Returns {@code true} if the table with the give name exists.
-   */
-  boolean tableExists(String tableId) throws IOException;
-
-  /**
-   * Returns a {@link Reader} that will read from the specified source.
-   */
-  Reader createReader(BigtableSource source) throws IOException;
-
-  /**
-   * Returns a {@link Writer} that will write to the specified table.
-   */
-  Writer openForWriting(String tableId) throws IOException;
-
-  /**
-   * Returns a set of row keys sampled from the underlying table. These contain information about
-   * the distribution of keys within the table.
-   */
-  List<SampleRowKeysResponse> getSampleRowKeys(BigtableSource source) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableServiceImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableServiceImpl.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableServiceImpl.java
deleted file mode 100644
index 9f32022..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableServiceImpl.java
+++ /dev/null
@@ -1,243 +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 com.google.cloud.dataflow.sdk.io.bigtable;
-
-import com.google.bigtable.admin.table.v1.GetTableRequest;
-import com.google.bigtable.v1.MutateRowRequest;
-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.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.async.AsyncExecutor;
-import com.google.cloud.bigtable.grpc.async.HeapSizeManager;
-import com.google.cloud.bigtable.grpc.scanner.ResultScanner;
-import com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO.BigtableSource;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.base.MoreObjects;
-import com.google.common.io.Closer;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Empty;
-
-import io.grpc.Status.Code;
-import io.grpc.StatusRuntimeException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable
- * service.
- */
-class BigtableServiceImpl implements BigtableService {
-  private static final Logger logger = LoggerFactory.getLogger(BigtableService.class);
-
-  public BigtableServiceImpl(BigtableOptions options) {
-    this.options = options;
-  }
-
-  private final BigtableOptions options;
-
-  @Override
-  public BigtableWriterImpl openForWriting(String tableId) throws IOException {
-    BigtableSession session = new BigtableSession(options);
-    String tableName = options.getClusterName().toTableNameStr(tableId);
-    return new BigtableWriterImpl(session, tableName);
-  }
-
-  @Override
-  public boolean tableExists(String tableId) throws IOException {
-    if (!BigtableSession.isAlpnProviderEnabled()) {
-      logger.info(
-          "Skipping existence check for table {} (BigtableOptions {}) because ALPN is not"
-              + " configured.",
-          tableId,
-          options);
-      return true;
-    }
-
-    try (BigtableSession session = new BigtableSession(options)) {
-      GetTableRequest getTable =
-          GetTableRequest.newBuilder()
-              .setName(options.getClusterName().toTableNameStr(tableId))
-              .build();
-      session.getTableAdminClient().getTable(getTable);
-      return true;
-    } catch (StatusRuntimeException e) {
-      if (e.getStatus().getCode() == Code.NOT_FOUND) {
-        return false;
-      }
-      String message =
-          String.format(
-              "Error checking whether table %s (BigtableOptions %s) exists", tableId, options);
-      logger.error(message, e);
-      throw new IOException(message, e);
-    }
-  }
-
-  private class BigtableReaderImpl implements Reader {
-    private BigtableSession session;
-    private final BigtableSource source;
-    private ResultScanner<Row> results;
-    private Row currentRow;
-
-    public BigtableReaderImpl(BigtableSession session, BigtableSource source) {
-      this.session = session;
-      this.source = source;
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      RowRange range =
-          RowRange.newBuilder()
-              .setStartKey(source.getRange().getStartKey().getValue())
-              .setEndKey(source.getRange().getEndKey().getValue())
-              .build();
-      ReadRowsRequest.Builder requestB =
-          ReadRowsRequest.newBuilder()
-              .setRowRange(range)
-              .setTableName(options.getClusterName().toTableNameStr(source.getTableId()));
-      if (source.getRowFilter() != null) {
-        requestB.setFilter(source.getRowFilter());
-      }
-      results = session.getDataClient().readRows(requestB.build());
-      return advance();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      currentRow = results.next();
-      return (currentRow != null);
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Goal: by the end of this function, both results and session are null and closed,
-      // independent of what errors they throw or prior state.
-
-      if (session == null) {
-        // Only possible when previously closed, so we know that results is also null.
-        return;
-      }
-
-      // Session does not implement Closeable -- it's AutoCloseable. So we can't register it with
-      // the Closer, but we can use the Closer to simplify the error handling.
-      try (Closer closer = Closer.create()) {
-        if (results != null) {
-          closer.register(results);
-          results = null;
-        }
-
-        session.close();
-      } finally {
-        session = null;
-      }
-    }
-
-    @Override
-    public Row getCurrentRow() throws NoSuchElementException {
-      if (currentRow == null) {
-        throw new NoSuchElementException();
-      }
-      return currentRow;
-    }
-  }
-
-  private static class BigtableWriterImpl implements Writer {
-    private BigtableSession session;
-    private AsyncExecutor executor;
-    private final MutateRowRequest.Builder partialBuilder;
-
-    public BigtableWriterImpl(BigtableSession session, String 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);
-    }
-
-    @Override
-    public void close() throws IOException {
-      try {
-        if (executor != null) {
-          executor.flush();
-          executor = null;
-        }
-      } finally {
-        if (session != null) {
-          session.close();
-          session = null;
-        }
-      }
-    }
-
-    @Override
-    public ListenableFuture<Empty> writeRecord(KV<ByteString, Iterable<Mutation>> record)
-        throws IOException {
-      MutateRowRequest r =
-          partialBuilder
-              .clone()
-              .setRowKey(record.getKey())
-              .addAllMutations(record.getValue())
-              .build();
-      try {
-        return executor.mutateRowAsync(r);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Write interrupted", e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects
-        .toStringHelper(BigtableServiceImpl.class)
-        .add("options", options)
-        .toString();
-  }
-
-  @Override
-  public Reader createReader(BigtableSource source) throws IOException {
-    BigtableSession session = new BigtableSession(options);
-    return new BigtableReaderImpl(session, source);
-  }
-
-  @Override
-  public List<SampleRowKeysResponse> getSampleRowKeys(BigtableSource source) throws IOException {
-    try (BigtableSession session = new BigtableSession(options)) {
-      SampleRowKeysRequest request =
-          SampleRowKeysRequest.newBuilder()
-              .setTableName(options.getClusterName().toTableNameStr(source.getTableId()))
-              .build();
-      return session.getDataClient().sampleRowKeys(request);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/package-info.java
deleted file mode 100644
index 553f46c..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-/**
- * Defines transforms for reading and writing from Google Cloud Bigtable.
- *
- * @see com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO
- */
-package com.google.cloud.dataflow.sdk.io.bigtable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java
deleted file mode 100644
index 5f0050d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java
+++ /dev/null
@@ -1,38 +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.
- */
-/**
- * Defines transforms for reading and writing common storage formats, including
- * {@link com.google.cloud.dataflow.sdk.io.AvroIO},
- * {@link com.google.cloud.dataflow.sdk.io.BigQueryIO}, and
- * {@link com.google.cloud.dataflow.sdk.io.TextIO}.
- *
- * <p>The classes in this package provide {@code Read} transforms that create PCollections
- * from existing storage:
- * <pre>{@code
- * PCollection<TableRow> inputData = pipeline.apply(
- *     BigQueryIO.Read.named("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"));
- * } </pre>
- */
-package com.google.cloud.dataflow.sdk.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKey.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKey.java
deleted file mode 100644
index 5b9a003..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKey.java
+++ /dev/null
@@ -1,174 +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 com.google.cloud.dataflow.sdk.io.range;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ByteString.ByteIterator;
-
-import java.io.Serializable;
-
-/**
- * A class representing a key consisting of an array of bytes. Arbitrary-length
- * {@code byte[]} keys are typical in key-value stores such as Google Cloud Bigtable.
- *
- * <p>Instances of {@link ByteKey} are immutable.
- *
- * <p>{@link ByteKey} implements {@link Comparable Comparable&lt;ByteKey&gt;} by comparing the
- * arrays in lexicographic order. The smallest {@link ByteKey} is a zero-length array; the successor
- * to a key is the same key with an additional 0 byte appended; and keys have unbounded size.
- *
- * <p>Note that the empty {@link ByteKey} compares smaller than all other keys, but some systems
- * have the semantic that when an empty {@link ByteKey} is used as an upper bound, it represents
- * the largest possible key. In these cases, implementors should use {@link #isEmpty} to test
- * whether an upper bound key is empty.
- */
-public final class ByteKey implements Comparable<ByteKey>, Serializable {
-  /** An empty key. */
-  public static final ByteKey EMPTY = ByteKey.of();
-
-  /**
-   * Creates a new {@link ByteKey} backed by the specified {@link ByteString}.
-   */
-  public static ByteKey of(ByteString value) {
-    return new ByteKey(value);
-  }
-
-  /**
-   * Creates a new {@link ByteKey} backed by a copy of the specified {@code byte[]}.
-   *
-   * <p>Makes a copy of the underlying array.
-   */
-  public static ByteKey copyFrom(byte[] bytes) {
-    return of(ByteString.copyFrom(bytes));
-  }
-
-  /**
-   * Creates a new {@link ByteKey} backed by a copy of the specified {@code int[]}. This method is
-   * primarily used as a convenience to create a {@link ByteKey} in code without casting down to
-   * signed Java {@link Byte bytes}:
-   *
-   * <pre>{@code
-   * ByteKey key = ByteKey.of(0xde, 0xad, 0xbe, 0xef);
-   * }</pre>
-   *
-   * <p>Makes a copy of the input.
-   */
-  public static ByteKey of(int... bytes) {
-    byte[] ret = new byte[bytes.length];
-    for (int i = 0; i < bytes.length; ++i) {
-      ret[i] = (byte) (bytes[i] & 0xff);
-    }
-    return ByteKey.copyFrom(ret);
-  }
-
-  /**
-   * Returns an immutable {@link ByteString} representing this {@link ByteKey}.
-   *
-   * <p>Does not copy.
-   */
-  public ByteString getValue() {
-    return value;
-  }
-
-  /**
-   * Returns a newly-allocated {@code byte[]} representing this {@link ByteKey}.
-   *
-   * <p>Copies the underlying {@code byte[]}.
-   */
-  public byte[] getBytes() {
-    return value.toByteArray();
-  }
-
-  /**
-   * Returns {@code true} if the {@code byte[]} backing this {@link ByteKey} is of length 0.
-   */
-  public boolean isEmpty() {
-    return value.isEmpty();
-  }
-
-  /**
-   * {@link ByteKey} implements {@link Comparable Comparable&lt;ByteKey&gt;} by comparing the
-   * arrays in lexicographic order. The smallest {@link ByteKey} is a zero-length array; the
-   * successor to a key is the same key with an additional 0 byte appended; and keys have unbounded
-   * size.
-   */
-  @Override
-  public int compareTo(ByteKey other) {
-    checkNotNull(other, "other");
-    ByteIterator thisIt = value.iterator();
-    ByteIterator otherIt = other.value.iterator();
-    while (thisIt.hasNext() && otherIt.hasNext()) {
-      // (byte & 0xff) converts [-128,127] bytes to [0,255] ints.
-      int cmp = (thisIt.nextByte() & 0xff) - (otherIt.nextByte() & 0xff);
-      if (cmp != 0) {
-        return cmp;
-      }
-    }
-    // If we get here, the prefix of both arrays is equal up to the shorter array. The array with
-    // more bytes is larger.
-    return value.size() - other.value.size();
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////
-  private final ByteString value;
-
-  private ByteKey(ByteString value) {
-    this.value = value;
-  }
-
-  /** Array used as a helper in {@link #toString}. */
-  private static final char[] HEX =
-      new char[] {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
-
-  // Prints the key as a string "[deadbeef]".
-  @Override
-  public String toString() {
-    char[] encoded = new char[2 * value.size() + 2];
-    encoded[0] = '[';
-    int cnt = 1;
-    ByteIterator iterator = value.iterator();
-    while (iterator.hasNext()) {
-      byte b = iterator.nextByte();
-      encoded[cnt] = HEX[(b & 0xF0) >>> 4];
-      ++cnt;
-      encoded[cnt] = HEX[b & 0xF];
-      ++cnt;
-    }
-    encoded[cnt] = ']';
-    return new String(encoded);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o == this) {
-      return true;
-    }
-    if (!(o instanceof ByteKey)) {
-      return false;
-    }
-    ByteKey other = (ByteKey) o;
-    return (other.value.size() == value.size()) && this.compareTo(other) == 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return value.hashCode();
-  }
-}


[57/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java
index 4997d9b..10259d9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.KV;
 
 import org.junit.Assert;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java
index 44571c7..1faf58f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static com.google.common.base.Preconditions.checkArgument;
+
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.instanceOf;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java
index 9494a56..b40457c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
index d978c3c..73fb464 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -62,7 +62,7 @@ public class DoubleCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAAAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
index 99452d9..e2daefc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.collect.Lists;
 
 import org.joda.time.Duration;
@@ -58,7 +59,7 @@ public class DurationCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/EntityCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/EntityCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/EntityCoderTest.java
index 8dd2926..2a22925 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/EntityCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/EntityCoderTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static com.google.api.services.datastore.client.DatastoreHelper.makeKey;
 import static com.google.api.services.datastore.client.DatastoreHelper.makeProperty;
 import static com.google.api.services.datastore.client.DatastoreHelper.makeValue;
 
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+
 import com.google.api.services.datastore.DatastoreV1.Entity;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -81,7 +82,7 @@ public class EntityCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAGwoZEhcKCFRlc3RLaW5kGgtlbXB0eUVudGl0eQ",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
index 5b225e3..c1ce7e6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.UnsignedBytes;
 
@@ -88,7 +89,7 @@ public class InstantCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "gAAAAAAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
index fd0212c..42397b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -84,7 +84,7 @@ public class IterableCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
index 26c1198..1a00417 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
index e0f4589..47fcad7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.KV;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.collect.ImmutableMap;
 
 import org.junit.Rule;
@@ -94,7 +95,7 @@ public class KvCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AP____8P",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
index 043f223..05b6681 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -91,7 +91,7 @@ public class ListCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
index 5eb16c2..b92b656 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+
 import com.google.common.collect.ImmutableMap;
 
 import org.junit.Rule;
@@ -83,7 +84,7 @@ public class MapCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
index 7628dff..1e669ac 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.theInstance;
@@ -24,7 +24,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.CoderProperties;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;
@@ -75,7 +76,7 @@ public class NullableCoderTest {
    * Generated data to check that the wire format has not changed. To regenerate, see
    * {@code PrintBase64Encodings}.
    *
-   * @see com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings
+   * @see org.apache.beam.sdk.coders.PrintBase64Encodings
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AQA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
index 77be235..9ab66d1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.CoderUtils;
 
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 
@@ -35,9 +36,9 @@ import java.util.List;
  * {@code
  *   mvn test-compile exec:java \
  *       -Dexec.classpathScope=test \
- *       -Dexec.mainClass=com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings
- *       -Dexec.args='com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoderTest.TEST_CODER \
- *           com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoderTest.TEST_VALUES'
+ *       -Dexec.mainClass=org.apache.beam.sdk.coders.PrintBase64Encodings
+ *       -Dexec.args='org.apache.beam.sdk.coders.BigEndianIntegerCoderTest.TEST_CODER \
+ *           org.apache.beam.sdk.coders.BigEndianIntegerCoderTest.TEST_VALUES'
  * }
  */
 public class PrintBase64Encodings {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/Proto2CoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/Proto2CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/Proto2CoderTest.java
index cd92cc5..65a392c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/Proto2CoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/Proto2CoderTest.java
@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
index 799da7c..98c4d6f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.Serializer;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.Serializer;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.Matchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
index 58dbd2b..e353504 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -62,7 +62,7 @@ public class SetCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAA",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java
index be92e9d..a3a0f45 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.CoderProperties;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -60,7 +60,7 @@ public class StandardCoderTest {
     @Override
     @Nullable
     public Boolean decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       int value = inStream.read();
       if (value == 0) {
@@ -123,7 +123,7 @@ public class StandardCoderTest {
     @Override
     @Nullable
     public ObjectIdentityBoolean decode(
-        InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
         throws CoderException, IOException {
       int value = inStream.read();
       if (value == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java
index 999193c..2a4f2e6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.testing.CoderProperties;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
index a6b7f96..5c810f7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -52,7 +52,7 @@ public class StringUtf8CoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java
index f28dfa2..f8f5d5d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java
index f13a9ab..03f2339 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -72,7 +73,7 @@ public class TableRowJsonCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "e30",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
index 01b90ae..252f8a4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -59,7 +59,7 @@ public class TextualIntegerCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "LTEx",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
index 8ac7076..3da242d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -59,7 +59,7 @@ public class VarIntCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "9f___w8",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
index 1ee15bc..4cd2216 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -61,7 +61,7 @@ public class VarLongCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "9f__________AQ",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java
index 025094d..b35adf6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders.protobuf;
+package org.apache.beam.sdk.coders.protobuf;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageWithMap;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java
index 3d7dafe..b41c340 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java
@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders.protobuf;
+package org.apache.beam.sdk.coders.protobuf;
+
+import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.checkProto2Syntax;
+import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.getRecursiveDescriptorsForClass;
+import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.verifyDeterministic;
 
-import static com.google.cloud.dataflow.sdk.coders.protobuf.ProtobufUtil.checkProto2Syntax;
-import static com.google.cloud.dataflow.sdk.coders.protobuf.ProtobufUtil.getRecursiveDescriptorsForClass;
-import static com.google.cloud.dataflow.sdk.coders.protobuf.ProtobufUtil.verifyDeterministic;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+
 import com.google.api.services.datastore.DatastoreV1.Entity;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA;
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b16aad1..f32a420 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
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipeline;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.DirectPipeline;
+import org.apache.beam.sdk.runners.DirectPipelineRunner.EvaluationResults;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f9e4dba..08f146f 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
@@ -23,15 +23,16 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO.Write.Bound;
-import com.google.cloud.dataflow.sdk.runners.DirectPipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+import org.apache.beam.sdk.io.AvroIO.Write.Bound;
+import org.apache.beam.sdk.runners.DirectPipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterators;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
index 775f5c3..c946115 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.AvroSource.AvroReader;
-import com.google.cloud.dataflow.sdk.io.AvroSource.AvroReader.Seeker;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+import org.apache.beam.sdk.io.AvroSource.AvroReader;
+import org.apache.beam.sdk.io.AvroSource.AvroReader.Seeker;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+
 import com.google.common.base.MoreObjects;
 
 import org.apache.avro.Schema;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0d084af..b9af1e2 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
@@ -15,31 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.BigQueryServices;
+import org.apache.beam.sdk.util.BigQueryServices.Status;
+import org.apache.beam.sdk.util.CoderUtils;
+
 import com.google.api.client.util.Data;
 import com.google.api.services.bigquery.model.JobConfigurationLoad;
 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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices.Status;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
index 24b0fca..094d597 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.dataflow.TestCountingSource;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
+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.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
index 331c896..a4b08f6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.instanceOf;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.io.CompressedSource.CompressionMode;
-import com.google.cloud.dataflow.sdk.io.CompressedSource.DecompressingChannelFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.CompressedSource.CompressionMode;
+import org.apache.beam.sdk.io.CompressedSource.DecompressingChannelFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.values.PCollection;
+
 import com.google.common.io.Files;
 import com.google.common.primitives.Bytes;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
index 2b369ed..b3b9b08 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
@@ -16,24 +16,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.CountingInput.UnboundedCountingInput;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput;
+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;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
index 5abd081..a261fb2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.lessThan;
@@ -23,24 +23,24 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.CountingSource.CounterMark;
-import com.google.cloud.dataflow.sdk.io.CountingSource.UnboundedCountingSource;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingSource.CounterMark;
+import org.apache.beam.sdk.io.CountingSource.UnboundedCountingSource;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+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;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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
index dcdb229..695fdeb 100644
--- 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.api.services.datastore.client.DatastoreHelper.makeKey;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.lessThanOrEqualTo;
@@ -35,6 +36,14 @@ 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.util.TestCredential;
+
 import com.google.api.services.datastore.DatastoreV1.Entity;
 import com.google.api.services.datastore.DatastoreV1.EntityResult;
 import com.google.api.services.datastore.DatastoreV1.Key;
@@ -49,13 +58,6 @@ import com.google.api.services.datastore.DatastoreV1.Value;
 import com.google.api.services.datastore.client.Datastore;
 import com.google.api.services.datastore.client.DatastoreHelper;
 import com.google.api.services.datastore.client.QuerySplitter;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO.DatastoreReader;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO.DatastoreWriter;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
 import com.google.common.collect.Lists;
 
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
index 44d59d4..0e434fc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
@@ -23,11 +23,11 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileResult;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation;
+import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention;
+import org.apache.beam.sdk.io.FileBasedSink.FileResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 1071a67..bedbc99 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
+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 com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -28,19 +29,20 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.FileBasedSource.FileBasedReader;
-import com.google.cloud.dataflow.sdk.io.Source.Reader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.IOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.FileBasedSource.FileBasedReader;
+import org.apache.beam.sdk.io.Source.Reader;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.util.CoderUtils;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
index 15b9b3f..e9b61aa 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
+import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource;
 
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f7d81fc..852ab7d 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
index 0564072..402eb52 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 6a660bf..09b7d22 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
@@ -15,35 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import static org.apache.beam.sdk.TestUtils.INTS_ARRAY;
+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 com.google.cloud.dataflow.sdk.TestUtils.INTS_ARRAY;
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_INTS_ARRAY;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 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 com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType;
-import com.google.cloud.dataflow.sdk.io.TextIO.TextSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.TextualIntegerCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.TextIO.CompressionType;
+import org.apache.beam.sdk.io.TextIO.TextSource;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.util.CoderUtils;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 da48772..865dbe1 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -24,20 +24,21 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest.TestPipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.Sink.WriteOperation;
+import org.apache.beam.sdk.io.Sink.Writer;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.PipelineOptionsFactoryTest.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.MoreObjects;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
index 3ba720b..b7407b0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
-import com.google.cloud.dataflow.sdk.io.XmlSink.XmlWriteOperation;
-import com.google.cloud.dataflow.sdk.io.XmlSink.XmlWriter;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.io.XmlSink.XmlWriteOperation;
+import org.apache.beam.sdk.io.XmlSink.XmlWriter;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
 import com.google.common.collect.Lists;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9865615..24ea70e 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 import static org.hamcrest.Matchers.both;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
@@ -27,13 +28,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.Source.Reader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.Source.Reader;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/bigtable/BigtableIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/bigtable/BigtableIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/bigtable/BigtableIOTest.java
index 7cb77c7..135fcb8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/bigtable/BigtableIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/bigtable/BigtableIOTest.java
@@ -15,19 +15,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.bigtable;
+package org.apache.beam.sdk.io.bigtable;
+
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSourcesEqualReferenceSource;
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
+import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSourcesEqualReferenceSource;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 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;
 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.bigtable.BigtableIO.BigtableSource;
+import org.apache.beam.sdk.io.range.ByteKey;
+import org.apache.beam.sdk.io.range.ByteKeyRange;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.bigtable.v1.Cell;
 import com.google.bigtable.v1.Column;
 import com.google.bigtable.v1.Family;
@@ -37,18 +52,6 @@ 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.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO.BigtableSource;
-import com.google.cloud.dataflow.sdk.io.range.ByteKey;
-import com.google.cloud.dataflow.sdk.io.range.ByteKeyRange;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java
index 5bdf254..50ec272 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.junit.Assert.assertThat;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
index 438e4e5..e801067 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.hamcrest.Matchers.closeTo;
 import static org.hamcrest.Matchers.containsString;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
index 4bcc5c6..f068ec9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.lessThan;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c5268fa..958fa48 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java
index 1eba3a0..b8c85ee 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static org.hamcrest.Matchers.lessThan;
 import static org.junit.Assert.assertEquals;



[18/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java
deleted file mode 100644
index c37d6c7..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java
+++ /dev/null
@@ -1,1685 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.api.client.json.JsonFactory;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.model.JobConfigurationLoad;
-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;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices.LoadService;
-import com.google.cloud.dataflow.sdk.util.BigQueryServicesImpl;
-import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter;
-import com.google.cloud.dataflow.sdk.util.BigQueryTableRowIterator;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Reshuffle;
-import com.google.cloud.dataflow.sdk.util.SystemDoFnInternal;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link PTransform}s for reading and writing
- * <a href="https://developers.google.com/bigquery/">BigQuery</a> tables.
- *
- * <h3>Table References</h3>
- * <p>A fully-qualified BigQuery table name consists of three components:
- * <ul>
- *   <li>{@code projectId}: the Cloud project id (defaults to
- *       {@link GcpOptions#getProject()}).
- *   <li>{@code datasetId}: the BigQuery dataset id, unique within a project.
- *   <li>{@code tableId}: a table id, unique within a dataset.
- * </ul>
- *
- * <p>BigQuery table references are stored as a {@link TableReference}, which comes
- * from the <a href="https://cloud.google.com/bigquery/client-libraries">
- * BigQuery Java Client API</a>.
- * Tables can be referred to as Strings, with or without the {@code projectId}.
- * A helper function is provided ({@link BigQueryIO#parseTableSpec(String)})
- * that parses the following string forms into a {@link TableReference}:
- *
- * <ul>
- *   <li>[{@code project_id}]:[{@code dataset_id}].[{@code table_id}]
- *   <li>[{@code dataset_id}].[{@code table_id}]
- * </ul>
- *
- * <h3>Reading</h3>
- * <p>To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation.
- * 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"));
- * }</pre>
- *
- * <p>See {@link TableRow} for more information on the {@link TableRow} object.
- *
- * <p>Users may provide a query to read from rather than reading all of a BigQuery table. If
- * specified, the result obtained by executing the specified query will be used as the data of the
- * input transform.
- *
- * <pre>{@code
- * PCollection<TableRow> shakespeare = pipeline.apply(
- *     BigQueryIO.Read.named("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.
- * Pipeline construction will fail with a validation error if neither or both are specified.
- *
- * <h3>Writing</h3>
- * <p>To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation.
- * This consumes a {@link PCollection} of {@link TableRow TableRows} as input.
- * <pre>{@code
- * PCollection<TableRow> quotes = ...
- *
- * List<TableFieldSchema> fields = new ArrayList<>();
- * fields.add(new TableFieldSchema().setName("source").setType("STRING"));
- * fields.add(new TableFieldSchema().setName("quote").setType("STRING"));
- * 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));
- * }</pre>
- *
- * <p>See {@link BigQueryIO.Write} for details on how to specify if a write should
- * append to an existing table, replace the table, or verify that the table is
- * empty. Note that the dataset being written to must already exist. Write
- * dispositions are not supported in streaming mode.
- *
- * <h3>Sharding BigQuery output tables</h3>
- * <p>A common use case is to dynamically generate BigQuery table names based on
- * the current window. To support this,
- * {@link BigQueryIO.Write#to(SerializableFunction)}
- * accepts a function mapping the current window to a tablespec. For example,
- * here's code that outputs daily tables to BigQuery:
- * <pre>{@code
- * 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) {
- *             // The cast below is safe because CalendarWindows.days(1) produces IntervalWindows.
- *             String dayString = DateTimeFormat.forPattern("yyyy_MM_dd")
- *                  .withZone(DateTimeZone.UTC)
- *                  .print(((IntervalWindow) window).start());
- *             return "my-project:output.output_table_" + dayString;
- *           }
- *         }));
- * }</pre>
- *
- * <p>Per-window tables are not yet supported in batch mode.
- *
- * <h3>Permissions</h3>
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding {@link PipelineRunner}s for
- * more details.
- *
- * <p>Please see <a href="https://cloud.google.com/bigquery/access-control">BigQuery Access Control
- * </a> for security and permission related information specific to BigQuery.
- */
-public class BigQueryIO {
-  private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class);
-
-  /**
-   * Singleton instance of the JSON factory used to read and write JSON
-   * formatted rows.
-   */
-  private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
-
-  /**
-   * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
-   * IDs must start with a letter and may not end with a dash.
-   * This regex isn't exact - this allows for patterns that would be rejected by
-   * the service, but this is sufficient for basic parsing of table references.
-   */
-  private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]";
-
-  /**
-   * Regular expression that matches Dataset IDs.
-   */
-  private static final String DATASET_REGEXP = "[-\\w.]{1,1024}";
-
-  /**
-   * Regular expression that matches Table IDs.
-   */
-  private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}";
-
-  /**
-   * Matches table specifications in the form {@code "[project_id]:[dataset_id].[table_id]"} or
-   * {@code "[dataset_id].[table_id]"}.
-   */
-  private static final String DATASET_TABLE_REGEXP =
-      String.format("((?<PROJECT>%s):)?(?<DATASET>%s)\\.(?<TABLE>%s)", PROJECT_ID_REGEXP,
-          DATASET_REGEXP, TABLE_REGEXP);
-
-  private static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP);
-
-  // TODO: make this private and remove improper access from BigQueryIOTranslator.
-  public static final String SET_PROJECT_FROM_OPTIONS_WARNING =
-      "No project specified for BigQuery table \"%1$s.%2$s\". Assuming it is in \"%3$s\". If the"
-      + " table is in a different project please specify it as a part of the BigQuery table"
-      + " definition.";
-
-  private static final String RESOURCE_NOT_FOUND_ERROR =
-      "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline"
-          + " execution. If the %1$s is created by an earlier stage of the pipeline, this"
-          + " validation can be disabled using #withoutValidation.";
-
-  private static final String UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR =
-      "Unable to confirm BigQuery %1$s presence for table \"%2$s\". If the %1$s is created by"
-          + " an earlier stage of the pipeline, this validation can be disabled using"
-          + " #withoutValidation.";
-
-  /**
-   * Parse a table specification in the form
-   * {@code "[project_id]:[dataset_id].[table_id]"} or {@code "[dataset_id].[table_id]"}.
-   *
-   * <p>If the project id is omitted, the default project id is used.
-   */
-  public static TableReference parseTableSpec(String tableSpec) {
-    Matcher match = TABLE_SPEC.matcher(tableSpec);
-    if (!match.matches()) {
-      throw new IllegalArgumentException(
-          "Table reference is not in [project_id]:[dataset_id].[table_id] "
-          + "format: " + tableSpec);
-    }
-
-    TableReference ref = new TableReference();
-    ref.setProjectId(match.group("PROJECT"));
-
-    return ref.setDatasetId(match.group("DATASET")).setTableId(match.group("TABLE"));
-  }
-
-  /**
-   * Returns a canonical string representation of the {@link TableReference}.
-   */
-  public static String toTableSpec(TableReference ref) {
-    StringBuilder sb = new StringBuilder();
-    if (ref.getProjectId() != null) {
-      sb.append(ref.getProjectId());
-      sb.append(":");
-    }
-
-    sb.append(ref.getDatasetId()).append('.').append(ref.getTableId());
-    return sb.toString();
-  }
-
-  /**
-   * A {@link PTransform} that reads from a BigQuery table and returns a
-   * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table.
-   *
-   * <p>Each {@link TableRow} contains values indexed by column name. Here is a
-   * sample processing function that processes a "line" column from rows:
-   * <pre>{@code
-   * static class ExtractWordsFn extends DoFn<TableRow, String> {
-   *   public void processElement(ProcessContext c) {
-   *     // Get the "line" field of the TableRow object, split it into words, and emit them.
-   *     TableRow row = c.element();
-   *     String[] words = row.get("line").toString().split("[^a-zA-Z']+");
-   *     for (String word : words) {
-   *       if (!word.isEmpty()) {
-   *         c.output(word);
-   *       }
-   *     }
-   *   }
-   * }}</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
-     * {@code "[dataset_id].[table_id]"} for tables within the current project.
-     */
-    public static Bound from(String tableSpec) {
-      return new Bound().from(tableSpec);
-    }
-
-    /**
-     * Reads results received after executing the given query.
-     */
-    public static Bound fromQuery(String query) {
-      return new Bound().fromQuery(query);
-    }
-
-    /**
-     * Reads a BigQuery table specified as a {@link TableReference} object.
-     */
-    public static Bound from(TableReference table) {
-      return new Bound().from(table);
-    }
-
-    /**
-     * Disables BigQuery table validation, which is enabled by default.
-     */
-    public static Bound withoutValidation() {
-      return new Bound().withoutValidation();
-    }
-
-    /**
-     * A {@link PTransform} that reads from a BigQuery table and returns a bounded
-     * {@link PCollection} of {@link TableRow TableRows}.
-     */
-    public static class Bound extends PTransform<PInput, PCollection<TableRow>> {
-      TableReference table;
-      final String query;
-      final boolean validate;
-      @Nullable
-      Boolean flattenResults;
-
-      private static final String QUERY_VALIDATION_FAILURE_ERROR =
-          "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
-          + " pipeline, This validation can be disabled using #withoutValidation.";
-
-      private Bound() {
-        this(null, null, null, true, null);
-      }
-
-      private Bound(String name, String query, TableReference reference, boolean validate,
-          Boolean flattenResults) {
-        super(name);
-        this.table = reference;
-        this.query = query;
-        this.validate = validate;
-        this.flattenResults = flattenResults;
-      }
-
-      /**
-       * 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, table, validate, flattenResults);
-      }
-
-      /**
-       * Returns a copy of this transform that reads from the specified table. Refer to
-       * {@link #parseTableSpec(String)} for the specification format.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound from(String tableSpec) {
-        return from(parseTableSpec(tableSpec));
-      }
-
-      /**
-       * Returns a copy of this transform that reads from the specified table.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound from(TableReference table) {
-        return new Bound(name, query, table, validate, flattenResults);
-      }
-
-      /**
-       * Returns a copy of this transform that reads the results of the specified query.
-       *
-       * <p>Does not modify this object.
-       *
-       * <p>By default, the query results will be flattened -- see
-       * "flattenResults" in the <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
-       * Jobs documentation</a> for more information.  To disable flattening, use
-       * {@link BigQueryIO.Read.Bound#withoutResultFlattening}.
-       */
-      public Bound fromQuery(String query) {
-        return new Bound(name, query, table, validate,
-            MoreObjects.firstNonNull(flattenResults, Boolean.TRUE));
-      }
-
-      /**
-       * Disable table validation.
-       */
-      public Bound withoutValidation() {
-        return new Bound(name, query, table, false, flattenResults);
-      }
-
-      /**
-       * Disable <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">
-       * flattening of query results</a>.
-       *
-       * <p>Only valid when a query is used ({@link #fromQuery}). Setting this option when reading
-       * from a table will cause an error during validation.
-       */
-      public Bound withoutResultFlattening() {
-        return new Bound(name, query, table, validate, false);
-      }
-
-      /**
-       * Validates the current {@link PTransform}.
-       */
-      @Override
-      public void validate(PInput input) {
-        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");
-        }
-
-        BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
-        if (table != null && table.getProjectId() == null) {
-          // If user does not specify a project we assume the table to be located in the project
-          // that owns the Dataflow job.
-          LOG.warn(String.format(SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
-              table.getTableId(), bqOptions.getProject()));
-          table.setProjectId(bqOptions.getProject());
-        }
-
-        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
-          // cases the withoutValidation method can be used to disable the check.
-          if (table != null) {
-            verifyDatasetPresence(bqOptions, table);
-            verifyTablePresence(bqOptions, table);
-          }
-          if (query != null) {
-            dryRunQuery(bqOptions, query);
-          }
-        }
-      }
-
-      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) {
-        return PCollection.<TableRow>createPrimitiveOutputInternal(
-            input.getPipeline(),
-            WindowingStrategy.globalDefault(),
-            IsBounded.BOUNDED)
-            // Force the output's Coder to be what the read is using, and
-            // unchangeable later, to ensure that we read the input in the
-            // format specified by the Read transform.
-            .setCoder(TableRowJsonCoder.of());
-      }
-
-      @Override
-      protected Coder<TableRow> getDefaultOutputCoder() {
-        return TableRowJsonCoder.of();
-      }
-
-      static {
-        DirectPipelineRunner.registerDefaultTransformEvaluator(
-            Bound.class, new DirectPipelineRunner.TransformEvaluator<Bound>() {
-              @Override
-              public void evaluate(
-                  Bound transform, DirectPipelineRunner.EvaluationContext context) {
-                evaluateReadHelper(transform, context);
-              }
-            });
-      }
-
-      /**
-       * Returns the table to write, or {@code null} if reading from a query instead.
-       */
-      public TableReference getTable() {
-        return table;
-      }
-
-      /**
-       * Returns the query to be read, or {@code null} if reading from a table instead.
-       */
-      public String getQuery() {
-        return query;
-      }
-
-      /**
-       * Returns true if table validation is enabled.
-       */
-      public boolean getValidate() {
-        return validate;
-      }
-
-      /**
-       * Returns true/false if result flattening is enabled/disabled, or null if not applicable.
-       */
-      public Boolean getFlattenResults() {
-        return flattenResults;
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Read() {}
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows}
-   * to a BigQuery table.
-   *
-   * <p>In BigQuery, each table has an encosing dataset. The dataset being written must already
-   * exist.
-   *
-   * <p>By default, tables will be created if they do not exist, which corresponds to a
-   * {@link CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's
-   * Jobs API. A schema must be provided (via {@link BigQueryIO.Write#withSchema(TableSchema)}),
-   * or else the transform may fail at runtime with an {@link IllegalArgumentException}.
-   *
-   * <p>By default, writes require an empty table, which corresponds to
-   * a {@link WriteDisposition#WRITE_EMPTY} disposition that matches the
-   * default of BigQuery's Jobs API.
-   *
-   * <p>Here is a sample transform that produces TableRow values containing
-   * "word" and "count" columns:
-   * <pre>{@code
-   * static class FormatCountsFn extends DoFn<KV<String, Long>, TableRow> {
-   *   public void processElement(ProcessContext c) {
-   *     TableRow row = new TableRow()
-   *         .set("word", c.element().getKey())
-   *         .set("count", c.element().getValue().intValue());
-   *     c.output(row);
-   *   }
-   * }}</pre>
-   */
-  public static class Write {
-    /**
-     * An enumeration type for the BigQuery create disposition strings.
-     *
-     * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.createDisposition">
-     * <code>configuration.query.createDisposition</code> in the BigQuery Jobs API</a>
-     */
-    public enum CreateDisposition {
-      /**
-       * Specifics that tables should not be created.
-       *
-       * <p>If the output table does not exist, the write fails.
-       */
-      CREATE_NEVER,
-
-      /**
-       * Specifies that tables should be created if needed. This is the default
-       * behavior.
-       *
-       * <p>Requires that a table schema is provided via {@link BigQueryIO.Write#withSchema}.
-       * This precondition is checked before starting a job. The schema is
-       * not required to match an existing table's schema.
-       *
-       * <p>When this transformation is executed, if the output table does not
-       * exist, the table is created from the provided schema. Note that even if
-       * the table exists, it may be recreated if necessary when paired with a
-       * {@link WriteDisposition#WRITE_TRUNCATE}.
-       */
-      CREATE_IF_NEEDED
-    }
-
-    /**
-     * An enumeration type for the BigQuery write disposition strings.
-     *
-     * @see <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.query.writeDisposition">
-     * <code>configuration.query.writeDisposition</code> in the BigQuery Jobs API</a>
-     */
-    public enum WriteDisposition {
-      /**
-       * Specifies that write should replace a table.
-       *
-       * <p>The replacement may occur in multiple steps - for instance by first
-       * removing the existing table, then creating a replacement, then filling
-       * it in. This is not an atomic operation, and external programs may
-       * see the table in any of these intermediate steps.
-       */
-      WRITE_TRUNCATE,
-
-      /**
-       * Specifies that rows may be appended to an existing table.
-       */
-      WRITE_APPEND,
-
-      /**
-       * Specifies that the output table must be empty. This is the default
-       * behavior.
-       *
-       * <p>If the output table is not empty, the write fails at runtime.
-       *
-       * <p>This check may occur long before data is written, and does not
-       * guarantee exclusive access to the table. If two programs are run
-       * concurrently, each specifying the same output table and
-       * a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is possible
-       * for both to succeed.
-       */
-      WRITE_EMPTY
-    }
-
-    /**
-     * 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.
-     */
-    public static Bound to(String tableSpec) {
-      return new Bound().to(tableSpec);
-    }
-
-    /** Creates a write transformation for the given table. */
-    public static Bound to(TableReference table) {
-      return new Bound().to(table);
-    }
-
-    /**
-     * Creates a write transformation from a function that maps windows to table specifications.
-     * Each time a new window is encountered, this function will be called and the resulting table
-     * will be created. Records within that window will be written to the associated table.
-     *
-     * <p>See {@link #parseTableSpec(String)} for the format that {@code tableSpecFunction} should
-     * return.
-     *
-     * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it should
-     * always return the same table specification.
-     */
-    public static Bound to(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
-      return new Bound().to(tableSpecFunction);
-    }
-
-    /**
-     * Creates a write transformation from a function that maps windows to {@link TableReference}
-     * objects.
-     *
-     * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
-     * always return the same table reference.
-     */
-    public static Bound toTableReference(
-        SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
-      return new Bound().toTableReference(tableRefFunction);
-    }
-
-    /**
-     * Creates a write transformation with the specified schema to use in table creation.
-     *
-     * <p>The schema is <i>required</i> only if writing to a table that does not already
-     * exist, and {@link CreateDisposition} is set to
-     * {@link CreateDisposition#CREATE_IF_NEEDED}.
-     */
-    public static Bound withSchema(TableSchema schema) {
-      return new Bound().withSchema(schema);
-    }
-
-    /** Creates a write transformation with the specified options for creating the table. */
-    public static Bound withCreateDisposition(CreateDisposition disposition) {
-      return new Bound().withCreateDisposition(disposition);
-    }
-
-    /** Creates a write transformation with the specified options for writing to the table. */
-    public static Bound withWriteDisposition(WriteDisposition disposition) {
-      return new Bound().withWriteDisposition(disposition);
-    }
-
-    /**
-     * Creates a write transformation with BigQuery table validation disabled.
-     */
-    public static Bound withoutValidation() {
-      return new Bound().withoutValidation();
-    }
-
-    /**
-     * A {@link PTransform} that can write either a bounded or unbounded
-     * {@link PCollection} of {@link TableRow TableRows} to a BigQuery table.
-     */
-    public static class Bound extends PTransform<PCollection<TableRow>, PDone> {
-      @Nullable final String jsonTableRef;
-
-      @Nullable final SerializableFunction<BoundedWindow, TableReference> tableRefFunction;
-
-      // Table schema. The schema is required only if the table does not exist.
-      @Nullable final String jsonSchema;
-
-      // Options for creating the table. Valid values are CREATE_IF_NEEDED and
-      // CREATE_NEVER.
-      final CreateDisposition createDisposition;
-
-      // Options for writing to the table. Valid values are WRITE_TRUNCATE,
-      // WRITE_APPEND and WRITE_EMPTY.
-      final WriteDisposition writeDisposition;
-
-      // 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;
-
-      private static class TranslateTableSpecFunction implements
-          SerializableFunction<BoundedWindow, TableReference> {
-        private SerializableFunction<BoundedWindow, String> tableSpecFunction;
-
-        TranslateTableSpecFunction(SerializableFunction<BoundedWindow, String> tableSpecFunction) {
-          this.tableSpecFunction = tableSpecFunction;
-        }
-
-        @Override
-        public TableReference apply(BoundedWindow value) {
-          return parseTableSpec(tableSpecFunction.apply(value));
-        }
-      }
-
-      /**
-       * @deprecated Should be private. Instead, use one of the factory methods in
-       * {@link BigQueryIO.Write}, such as {@link BigQueryIO.Write#to(String)}, to create an
-       * instance of this class.
-       */
-      @Deprecated
-      public Bound() {
-        this(null, null, null, null, CreateDisposition.CREATE_IF_NEEDED,
-            WriteDisposition.WRITE_EMPTY, true, null);
-      }
-
-      private Bound(String name, @Nullable String jsonTableRef,
-          @Nullable SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
-          @Nullable String jsonSchema,
-          CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate,
-          @Nullable BigQueryServices testBigQueryServices) {
-        super(name);
-        this.jsonTableRef = jsonTableRef;
-        this.tableRefFunction = tableRefFunction;
-        this.jsonSchema = jsonSchema;
-        this.createDisposition = checkNotNull(createDisposition, "createDisposition");
-        this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
-        this.validate = validate;
-        this.testBigQueryServices = testBigQueryServices;
-      }
-
-      /**
-       * 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.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound to(String tableSpec) {
-        return to(parseTableSpec(tableSpec));
-      }
-
-      /**
-       * Returns a copy of this write transformation, but writing to the specified table.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound to(TableReference table) {
-        return new Bound(name, toJsonString(table), tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
-       * Returns a copy of this write transformation, but using the specified function to determine
-       * which table to write to for each window.
-       *
-       * <p>Does not modify this object.
-       *
-       * <p>{@code tableSpecFunction} should be deterministic. When given the same window, it
-       * should always return the same table specification.
-       */
-      public Bound to(
-          SerializableFunction<BoundedWindow, String> tableSpecFunction) {
-        return toTableReference(new TranslateTableSpecFunction(tableSpecFunction));
-      }
-
-      /**
-       * Returns a copy of this write transformation, but using the specified function to determine
-       * which table to write to for each window.
-       *
-       * <p>Does not modify this object.
-       *
-       * <p>{@code tableRefFunction} should be deterministic. When given the same window, it should
-       * always return the same table reference.
-       */
-      public Bound toTableReference(
-          SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
-       * Returns a copy of this write transformation, but using the specified schema for rows
-       * to be written.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound withSchema(TableSchema schema) {
-        return new Bound(name, jsonTableRef, tableRefFunction, toJsonString(schema),
-            createDisposition, writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
-       * Returns a copy of this write transformation, but using the specified create disposition.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound withCreateDisposition(CreateDisposition createDisposition) {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
-       * Returns a copy of this write transformation, but using the specified write disposition.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound withWriteDisposition(WriteDisposition writeDisposition) {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
-       * Returns a copy of this write transformation, but without BigQuery table validation.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound withoutValidation() {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, false, testBigQueryServices);
-      }
-
-      @VisibleForTesting
-      Bound withTestServices(BigQueryServices testServices) {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testServices);
-      }
-
-      private static void verifyTableEmpty(
-          BigQueryOptions options,
-          TableReference table) {
-        try {
-          Bigquery client = Transport.newBigQueryClient(options).build();
-          BigQueryTableInserter inserter = new BigQueryTableInserter(client);
-          if (!inserter.isEmpty(table)) {
-            throw new IllegalArgumentException(
-                "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table));
-          }
-        } catch (IOException e) {
-          ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-          if (errorExtractor.itemNotFound(e)) {
-            // Nothing to do. If the table does not exist, it is considered empty.
-          } else {
-            throw new RuntimeException(
-                "unable to confirm BigQuery table emptiness for table "
-                    + BigQueryIO.toTableSpec(table), e);
-          }
-        }
-      }
-
-      @Override
-      public PDone apply(PCollection<TableRow> input) {
-        BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
-        TableReference table = getTable();
-        if (table == null && tableRefFunction == null) {
-          throw new IllegalStateException(
-              "must set the table reference of a BigQueryIO.Write transform");
-        }
-        if (table != null && tableRefFunction != null) {
-          throw new IllegalStateException(
-              "Cannot set both a table reference and a table function for a BigQueryIO.Write "
-                + "transform");
-        }
-
-        if (createDisposition == CreateDisposition.CREATE_IF_NEEDED && jsonSchema == null) {
-          throw new IllegalArgumentException("CreateDisposition is CREATE_IF_NEEDED, "
-              + "however no schema was provided.");
-        }
-
-        if (table != null && table.getProjectId() == null) {
-          // If user does not specify a project we assume the table to be located in the project
-          // that owns the Dataflow job.
-          String projectIdFromOptions = options.getProject();
-          LOG.warn(String.format(BigQueryIO.SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
-              table.getTableId(), projectIdFromOptions));
-          table.setProjectId(projectIdFromOptions);
-        }
-
-        // Check for destination table presence and emptiness for early failure notification.
-        // Note that a presence check can fail if the table or dataset are created by earlier stages
-        // of the pipeline. For these cases the withoutValidation method can be used to disable
-        // the check.
-        // Unfortunately we can't validate anything early in case tableRefFunction is specified.
-        if (table != null && validate) {
-          verifyDatasetPresence(options, table);
-          if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) {
-            verifyTablePresence(options, table);
-          }
-          if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) {
-            verifyTableEmpty(options, table);
-          }
-        }
-
-        // In streaming, BigQuery write is taken care of by StreamWithDeDup transform.
-        // We also currently do this if a tablespec function is specified.
-        if (options.isStreaming() || tableRefFunction != null) {
-          if (createDisposition == CreateDisposition.CREATE_NEVER) {
-            throw new IllegalArgumentException("CreateDispostion.CREATE_NEVER is not "
-                + "supported for unbounded PCollections or when using tablespec functions.");
-          }
-
-          if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) {
-            throw new IllegalArgumentException("WriteDisposition.WRITE_TRUNCATE is not "
-                + "supported for unbounded PCollections or when using tablespec functions.");
-          }
-
-          return input.apply(new StreamWithDeDup(table, tableRefFunction, getSchema()));
-        }
-
-        String tempLocation = options.getTempLocation();
-        checkArgument(!Strings.isNullOrEmpty(tempLocation),
-            "BigQueryIO.Write needs a GCS temp location to store temp files.");
-        if (testBigQueryServices == null) {
-          try {
-            GcsPath.fromUri(tempLocation);
-          } catch (IllegalArgumentException e) {
-            throw new IllegalArgumentException(String.format(
-                "BigQuery temp location expected a valid 'gs://' path, but was given '%s'",
-                tempLocation), e);
-          }
-        }
-        String jobIdToken = UUID.randomUUID().toString();
-        String tempFilePrefix = tempLocation + "/BigQuerySinkTemp/" + jobIdToken;
-        BigQueryServices bqServices = getBigQueryServices();
-        return input.apply("Write", com.google.cloud.dataflow.sdk.io.Write.to(
-            new BigQuerySink(
-                jobIdToken,
-                jsonTableRef,
-                jsonSchema,
-                getWriteDisposition(),
-                getCreateDisposition(),
-                tempFilePrefix,
-                input.getCoder(),
-                bqServices)));
-      }
-
-      @Override
-      protected Coder<Void> getDefaultOutputCoder() {
-        return VoidCoder.of();
-      }
-
-      /** Returns the create disposition. */
-      public CreateDisposition getCreateDisposition() {
-        return createDisposition;
-      }
-
-      /** Returns the write disposition. */
-      public WriteDisposition getWriteDisposition() {
-        return writeDisposition;
-      }
-
-      /** Returns the table schema. */
-      public TableSchema getSchema() {
-        return fromJsonString(jsonSchema, TableSchema.class);
-      }
-
-      /** Returns the table reference, or {@code null} if a . */
-      public TableReference getTable() {
-        return fromJsonString(jsonTableRef, TableReference.class);
-      }
-
-      /** Returns {@code true} if table validation is enabled. */
-      public boolean getValidate() {
-        return validate;
-      }
-
-      private BigQueryServices getBigQueryServices() {
-        if (testBigQueryServices != null) {
-          return testBigQueryServices;
-        } else {
-          return new BigQueryServicesImpl();
-        }
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Write() {}
-  }
-
-  /**
-   * {@link BigQuerySink} is implemented as a {@link FileBasedSink}.
-   *
-   * <p>It uses BigQuery load job to import files into BigQuery.
-   */
-  static class BigQuerySink extends FileBasedSink<TableRow> {
-    private final String jobIdToken;
-    @Nullable private final String jsonTable;
-    @Nullable private final String jsonSchema;
-    private final WriteDisposition writeDisposition;
-    private final CreateDisposition createDisposition;
-    private final Coder<TableRow> coder;
-    private final BigQueryServices bqServices;
-
-    public BigQuerySink(
-        String jobIdToken,
-        @Nullable String jsonTable,
-        @Nullable String jsonSchema,
-        WriteDisposition writeDisposition,
-        CreateDisposition createDisposition,
-        String tempFile,
-        Coder<TableRow> coder,
-        BigQueryServices bqServices) {
-      super(tempFile, ".json");
-      this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken");
-      this.jsonTable = jsonTable;
-      this.jsonSchema = jsonSchema;
-      this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
-      this.createDisposition = checkNotNull(createDisposition, "createDisposition");
-      this.coder = checkNotNull(coder, "coder");
-      this.bqServices = checkNotNull(bqServices, "bqServices");
-     }
-
-    @Override
-    public FileBasedSink.FileBasedWriteOperation<TableRow> createWriteOperation(
-        PipelineOptions options) {
-      return new BigQueryWriteOperation(this);
-    }
-
-    private static class BigQueryWriteOperation extends FileBasedWriteOperation<TableRow> {
-      // The maximum number of retry load jobs.
-      private static final int MAX_RETRY_LOAD_JOBS = 3;
-
-      private final BigQuerySink bigQuerySink;
-
-      private BigQueryWriteOperation(BigQuerySink sink) {
-        super(checkNotNull(sink, "sink"));
-        this.bigQuerySink = sink;
-      }
-
-      @Override
-      public FileBasedWriter<TableRow> createWriter(PipelineOptions options) throws Exception {
-        return new TableRowWriter(this, bigQuerySink.coder);
-      }
-
-      @Override
-      public void finalize(Iterable<FileResult> writerResults, PipelineOptions options)
-          throws IOException, InterruptedException {
-        try {
-          BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
-          List<String> tempFiles = Lists.newArrayList();
-          for (FileResult result : writerResults) {
-            tempFiles.add(result.getFilename());
-          }
-          if (!tempFiles.isEmpty()) {
-              load(
-                  bigQuerySink.bqServices.getLoadService(bqOptions),
-                  bigQuerySink.jobIdToken,
-                  fromJsonString(bigQuerySink.jsonTable, TableReference.class),
-                  tempFiles,
-                  fromJsonString(bigQuerySink.jsonSchema, TableSchema.class),
-                  bigQuerySink.writeDisposition,
-                  bigQuerySink.createDisposition);
-          }
-        } finally {
-          removeTemporaryFiles(options);
-        }
-      }
-
-      /**
-       * Import files into BigQuery with load jobs.
-       *
-       * <p>Returns if files are successfully loaded into BigQuery.
-       * Throws a RuntimeException if:
-       *     1. The status of one load job is UNKNOWN. This is to avoid duplicating data.
-       *     2. It exceeds {@code MAX_RETRY_LOAD_JOBS}.
-       *
-       * <p>If a load job failed, it will try another load job with a different job id.
-       */
-      private void load(
-          LoadService loadService,
-          String jobIdPrefix,
-          TableReference ref,
-          List<String> gcsUris,
-          @Nullable TableSchema schema,
-          WriteDisposition writeDisposition,
-          CreateDisposition createDisposition) throws InterruptedException, IOException {
-        JobConfigurationLoad loadConfig = new JobConfigurationLoad();
-        loadConfig.setSourceUris(gcsUris);
-        loadConfig.setDestinationTable(ref);
-        loadConfig.setSchema(schema);
-        loadConfig.setWriteDisposition(writeDisposition.name());
-        loadConfig.setCreateDisposition(createDisposition.name());
-        loadConfig.setSourceFormat("NEWLINE_DELIMITED_JSON");
-
-        boolean retrying = false;
-        String projectId = ref.getProjectId();
-        for (int i = 0; i < MAX_RETRY_LOAD_JOBS; ++i) {
-          String jobId = jobIdPrefix + "-" + i;
-          if (retrying) {
-            LOG.info("Previous load jobs failed, retrying.");
-          }
-          LOG.info("Starting BigQuery load job: {}", jobId);
-          loadService.startLoadJob(jobId, loadConfig);
-          BigQueryServices.Status jobStatus = loadService.pollJobStatus(projectId, jobId);
-          switch (jobStatus) {
-            case SUCCEEDED:
-              return;
-            case UNKNOWN:
-              throw new RuntimeException("Failed to poll the load job status.");
-            case FAILED:
-              LOG.info("BigQuery load job failed: {}", jobId);
-              retrying = true;
-              continue;
-            default:
-              throw new IllegalStateException("Unexpected job status: " + jobStatus);
-          }
-        }
-        throw new RuntimeException(
-            "Failed to create the load job, reached max retries: " + MAX_RETRY_LOAD_JOBS);
-      }
-    }
-
-    private static class TableRowWriter extends FileBasedWriter<TableRow> {
-      private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
-      private final Coder<TableRow> coder;
-      private OutputStream out;
-
-      public TableRowWriter(
-          FileBasedWriteOperation<TableRow> writeOperation, Coder<TableRow> coder) {
-        super(writeOperation);
-        this.mimeType = MimeTypes.TEXT;
-        this.coder = coder;
-      }
-
-      @Override
-      protected void prepareWrite(WritableByteChannel channel) throws Exception {
-        out = Channels.newOutputStream(channel);
-      }
-
-      @Override
-      public void write(TableRow value) throws Exception {
-        // Use Context.OUTER to encode and NEWLINE as the delimeter.
-        coder.encode(value, out, Context.OUTER);
-        out.write(NEWLINE);
-      }
-    }
-  }
-
-  private static void verifyDatasetPresence(BigQueryOptions options, 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));
-    } catch (Exception e) {
-      ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-      if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
-        throw new IllegalArgumentException(
-            String.format(RESOURCE_NOT_FOUND_ERROR, "dataset", BigQueryIO.toTableSpec(table)),
-            e);
-      } else {
-        throw new RuntimeException(
-            String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "dataset",
-                BigQueryIO.toTableSpec(table)),
-            e);
-      }
-    }
-  }
-
-  private static void verifyTablePresence(BigQueryOptions options, 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));
-    } catch (Exception e) {
-      ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-      if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
-        throw new IllegalArgumentException(
-            String.format(RESOURCE_NOT_FOUND_ERROR, "table", BigQueryIO.toTableSpec(table)), e);
-      } else {
-        throw new RuntimeException(
-            String.format(UNABLE_TO_CONFIRM_PRESENCE_OF_RESOURCE_ERROR, "table",
-                BigQueryIO.toTableSpec(table)),
-            e);
-      }
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Implementation of DoFn to perform streaming BigQuery write.
-   */
-  @SystemDoFnInternal
-  private static class StreamingWriteFn
-      extends DoFn<KV<ShardedKey<String>, TableRowInfo>, Void> {
-    /** TableSchema in JSON. Use String to make the class Serializable. */
-    private final String jsonTableSchema;
-
-    /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */
-    private transient Map<String, List<TableRow>> tableRows;
-
-    /** The list of unique ids for each BigQuery table row. */
-    private transient Map<String, List<String>> uniqueIdsForTableRows;
-
-    /** The list of tables created so far, so we don't try the creation
-        each time. */
-    private static Set<String> createdTables =
-        Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
-
-    /** Tracks bytes written, exposed as "ByteCount" Counter. */
-    private Aggregator<Long, Long> byteCountAggregator =
-        createAggregator("ByteCount", new Sum.SumLongFn());
-
-    /** Constructor. */
-    StreamingWriteFn(TableSchema schema) {
-      jsonTableSchema = toJsonString(schema);
-    }
-
-    /** Prepares a target BigQuery table. */
-    @Override
-    public void startBundle(Context context) {
-      tableRows = new HashMap<>();
-      uniqueIdsForTableRows = new HashMap<>();
-    }
-
-    /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */
-    @Override
-    public void processElement(ProcessContext context) {
-      String tableSpec = context.element().getKey().getKey();
-      List<TableRow> rows = getOrCreateMapListValue(tableRows, tableSpec);
-      List<String> uniqueIds = getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec);
-
-      rows.add(context.element().getValue().tableRow);
-      uniqueIds.add(context.element().getValue().uniqueId);
-    }
-
-    /** Writes the accumulated rows into BigQuery with streaming API. */
-    @Override
-    public void finishBundle(Context context) throws Exception {
-      BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class);
-      Bigquery client = Transport.newBigQueryClient(options).build();
-
-      for (String tableSpec : tableRows.keySet()) {
-        TableReference tableReference = getOrCreateTable(options, tableSpec);
-        flushRows(client, tableReference, tableRows.get(tableSpec),
-            uniqueIdsForTableRows.get(tableSpec));
-      }
-      tableRows.clear();
-      uniqueIdsForTableRows.clear();
-    }
-
-    public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec)
-        throws IOException {
-      TableReference tableReference = parseTableSpec(tableSpec);
-      if (!createdTables.contains(tableSpec)) {
-        synchronized (createdTables) {
-          // Another thread may have succeeded in creating the table in the meanwhile, so
-          // check again. This check isn't needed for correctness, but we add it to prevent
-          // every thread from attempting a create and overwhelming our BigQuery quota.
-          if (!createdTables.contains(tableSpec)) {
-            TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class);
-            Bigquery client = Transport.newBigQueryClient(options).build();
-            BigQueryTableInserter inserter = new BigQueryTableInserter(client);
-            inserter.getOrCreateTable(tableReference, WriteDisposition.WRITE_APPEND,
-                CreateDisposition.CREATE_IF_NEEDED, tableSchema);
-            createdTables.add(tableSpec);
-          }
-        }
-      }
-      return tableReference;
-    }
-
-    /** Writes the accumulated rows into BigQuery with streaming API. */
-    private void flushRows(Bigquery client, TableReference tableReference,
-        List<TableRow> tableRows, List<String> uniqueIds) {
-      if (!tableRows.isEmpty()) {
-        try {
-          BigQueryTableInserter inserter = new BigQueryTableInserter(client);
-          inserter.insertAll(tableReference, tableRows, uniqueIds, byteCountAggregator);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    }
-  }
-
-  private static class ShardedKey<K> {
-    private final K key;
-    private final int shardNumber;
-
-    public static <K> ShardedKey<K> of(K key, int shardNumber) {
-      return new ShardedKey<K>(key, shardNumber);
-    }
-
-    private ShardedKey(K key, int shardNumber) {
-      this.key = key;
-      this.shardNumber = shardNumber;
-    }
-
-    public K getKey() {
-      return key;
-    }
-
-    public int getShardNumber() {
-      return shardNumber;
-    }
-  }
-
-  /**
-   * A {@link Coder} for {@link ShardedKey}, using a wrapped key {@link Coder}.
-   */
-  private static class ShardedKeyCoder<KeyT>
-      extends StandardCoder<ShardedKey<KeyT>> {
-    public static <KeyT> ShardedKeyCoder<KeyT> of(Coder<KeyT> keyCoder) {
-      return new ShardedKeyCoder<>(keyCoder);
-    }
-
-    @JsonCreator
-    public static <KeyT> ShardedKeyCoder<KeyT> of(
-         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-        List<Coder<KeyT>> components) {
-      checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
-      return of(components.get(0));
-    }
-
-    protected ShardedKeyCoder(Coder<KeyT> keyCoder) {
-      this.keyCoder = keyCoder;
-      this.shardNumberCoder = VarIntCoder.of();
-    }
-
-    @Override
-    public List<? extends Coder<?>> getCoderArguments() {
-      return Arrays.asList(keyCoder);
-    }
-
-    @Override
-    public void encode(ShardedKey<KeyT> key, OutputStream outStream, Context context)
-        throws IOException {
-      keyCoder.encode(key.getKey(), outStream, context.nested());
-      shardNumberCoder.encode(key.getShardNumber(), outStream, context);
-    }
-
-    @Override
-    public ShardedKey<KeyT> decode(InputStream inStream, Context context)
-        throws IOException {
-      return new ShardedKey<KeyT>(
-          keyCoder.decode(inStream, context.nested()),
-          shardNumberCoder.decode(inStream, context));
-    }
-
-    @Override
-    public void verifyDeterministic() throws NonDeterministicException {
-      keyCoder.verifyDeterministic();
-    }
-
-    Coder<KeyT> keyCoder;
-    VarIntCoder shardNumberCoder;
-  }
-
-  private static class TableRowInfoCoder extends AtomicCoder<TableRowInfo> {
-    private static final TableRowInfoCoder INSTANCE = new TableRowInfoCoder();
-
-    @JsonCreator
-    public static TableRowInfoCoder of() {
-      return INSTANCE;
-    }
-
-    @Override
-    public void encode(TableRowInfo value, OutputStream outStream, Context context)
-      throws IOException {
-      if (value == null) {
-        throw new CoderException("cannot encode a null value");
-      }
-      tableRowCoder.encode(value.tableRow, outStream, context.nested());
-      idCoder.encode(value.uniqueId, outStream, context.nested());
-    }
-
-    @Override
-    public TableRowInfo decode(InputStream inStream, Context context)
-      throws IOException {
-      return new TableRowInfo(
-          tableRowCoder.decode(inStream, context.nested()),
-          idCoder.decode(inStream, context.nested()));
-    }
-
-    @Override
-    public void verifyDeterministic() throws NonDeterministicException {
-      throw new NonDeterministicException(this, "TableRows are not deterministic.");
-    }
-
-    TableRowJsonCoder tableRowCoder = TableRowJsonCoder.of();
-    StringUtf8Coder idCoder = StringUtf8Coder.of();
-  }
-
-  private static class TableRowInfo {
-    TableRowInfo(TableRow tableRow, String uniqueId) {
-      this.tableRow = tableRow;
-      this.uniqueId = uniqueId;
-    }
-
-    final TableRow tableRow;
-    final String uniqueId;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Fn that tags each table row with a unique id and destination table.
-   * To avoid calling UUID.randomUUID() for each element, which can be costly,
-   * a randomUUID is generated only once per bucket of data. The actual unique
-   * id is created by concatenating this randomUUID with a sequential number.
-   */
-  private static class TagWithUniqueIdsAndTable
-      extends DoFn<TableRow, KV<ShardedKey<String>, TableRowInfo>>
-      implements DoFn.RequiresWindowAccess {
-    /** TableSpec to write to. */
-    private final String tableSpec;
-
-    /** User function mapping windows to {@link TableReference} in JSON. */
-    private final SerializableFunction<BoundedWindow, TableReference> tableRefFunction;
-
-    private transient String randomUUID;
-    private transient long sequenceNo = 0L;
-
-    TagWithUniqueIdsAndTable(BigQueryOptions options, TableReference table,
-        SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
-      checkArgument(table == null ^ tableRefFunction == null,
-          "Exactly one of table or tableRefFunction should be set");
-      if (table != null) {
-        if (table.getProjectId() == null) {
-          table.setProjectId(options.as(BigQueryOptions.class).getProject());
-        }
-        this.tableSpec = toTableSpec(table);
-      } else {
-        tableSpec = null;
-      }
-      this.tableRefFunction = tableRefFunction;
-    }
-
-
-    @Override
-    public void startBundle(Context context) {
-      randomUUID = UUID.randomUUID().toString();
-    }
-
-    /** Tag the input with a unique id. */
-    @Override
-    public void processElement(ProcessContext context) throws IOException {
-      String uniqueId = randomUUID + sequenceNo++;
-      ThreadLocalRandom randomGenerator = ThreadLocalRandom.current();
-      String tableSpec = tableSpecFromWindow(
-          context.getPipelineOptions().as(BigQueryOptions.class), context.window());
-      // We output on keys 0-50 to ensure that there's enough batching for
-      // BigQuery.
-      context.output(KV.of(ShardedKey.of(tableSpec, randomGenerator.nextInt(0, 50)),
-          new TableRowInfo(context.element(), uniqueId)));
-    }
-
-    private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window) {
-      if (tableSpec != null) {
-        return tableSpec;
-      } else {
-        TableReference table = tableRefFunction.apply(window);
-        if (table.getProjectId() == null) {
-          table.setProjectId(options.getProject());
-        }
-        return toTableSpec(table);
-      }
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-  * PTransform that performs streaming BigQuery write. To increase consistency,
-  * it leverages BigQuery best effort de-dup mechanism.
-   */
-  private static class StreamWithDeDup extends PTransform<PCollection<TableRow>, PDone> {
-    private final transient TableReference tableReference;
-    private final SerializableFunction<BoundedWindow, TableReference> tableRefFunction;
-    private final transient TableSchema tableSchema;
-
-    /** Constructor. */
-    StreamWithDeDup(TableReference tableReference,
-        SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
-        TableSchema tableSchema) {
-      this.tableReference = tableReference;
-      this.tableRefFunction = tableRefFunction;
-      this.tableSchema = tableSchema;
-    }
-
-    @Override
-    protected Coder<Void> getDefaultOutputCoder() {
-      return VoidCoder.of();
-    }
-
-    @Override
-    public PDone apply(PCollection<TableRow> input) {
-      // A naive implementation would be to simply stream data directly to BigQuery.
-      // However, this could occasionally lead to duplicated data, e.g., when
-      // a VM that runs this code is restarted and the code is re-run.
-
-      // The above risk is mitigated in this implementation by relying on
-      // BigQuery built-in best effort de-dup mechanism.
-
-      // To use this mechanism, each input TableRow is tagged with a generated
-      // unique id, which is then passed to BigQuery and used to ignore duplicates.
-
-      PCollection<KV<ShardedKey<String>, TableRowInfo>> tagged = input.apply(ParDo.of(
-          new TagWithUniqueIdsAndTable(input.getPipeline().getOptions().as(BigQueryOptions.class),
-              tableReference, tableRefFunction)));
-
-      // To prevent having the same TableRow processed more than once with regenerated
-      // different unique ids, this implementation relies on "checkpointing", which is
-      // achieved as a side effect of having StreamingWriteFn immediately follow a GBK,
-      // performed by Reshuffle.
-      tagged
-          .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of()))
-          .apply(Reshuffle.<ShardedKey<String>, TableRowInfo>of())
-          .apply(ParDo.of(new StreamingWriteFn(tableSchema)));
-
-      // Note that the implementation to return PDone here breaks the
-      // implicit assumption about the job execution order. If a user
-      // implements a PTransform that takes PDone returned here as its
-      // input, the transform may not necessarily be executed after
-      // the BigQueryIO.Write.
-
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  private static String toJsonString(Object item) {
-    if (item == null) {
-      return null;
-    }
-    try {
-      return JSON_FACTORY.toString(item);
-    } catch (IOException e) {
-      throw new RuntimeException(
-          String.format("Cannot serialize %s to a JSON string.", item.getClass().getSimpleName()),
-          e);
-    }
-  }
-
-  private static <T> T fromJsonString(String json, Class<T> clazz) {
-    if (json == null) {
-      return null;
-    }
-    try {
-      return JSON_FACTORY.fromString(json, clazz);
-    } catch (IOException e) {
-      throw new RuntimeException(
-          String.format("Cannot deserialize %s from a JSON string: %s.", clazz, json),
-          e);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Disallow construction of utility class. */
-  private BigQueryIO() {}
-
-  /**
-   * Direct mode read evaluator.
-   *
-   * <p>This loads the entire table into an in-memory PCollection.
-   */
-  private static void evaluateReadHelper(
-      Read.Bound transform, DirectPipelineRunner.EvaluationContext context) {
-    BigQueryOptions options = context.getPipelineOptions();
-    Bigquery client = Transport.newBigQueryClient(options).build();
-    if (transform.table != null && transform.table.getProjectId() == null) {
-      transform.table.setProjectId(options.getProject());
-    }
-
-    BigQueryTableRowIterator iterator;
-    if (transform.query != null) {
-      LOG.info("Reading from BigQuery query {}", transform.query);
-      iterator =
-          BigQueryTableRowIterator.fromQuery(
-              transform.query, options.getProject(), client, transform.getFlattenResults());
-    } else {
-      LOG.info("Reading from BigQuery table {}", toTableSpec(transform.table));
-      iterator = BigQueryTableRowIterator.fromTable(transform.table, client);
-    }
-
-    try (BigQueryTableRowIterator ignored = iterator) {
-      List<TableRow> elems = new ArrayList<>();
-      iterator.open();
-      while (iterator.advance()) {
-        elems.add(iterator.getCurrent());
-      }
-      LOG.info("Number of records read from BigQuery: {}", elems.size());
-      context.setPCollection(context.getOutput(transform), elems);
-    } catch (IOException | InterruptedException e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new RuntimeException(e);
-    }
-  }
-
-  private static <K, V> List<V> getOrCreateMapListValue(Map<K, List<V>> map, K key) {
-    List<V> value = map.get(key);
-    if (value == null) {
-      value = new ArrayList<>();
-      map.put(key, value);
-    }
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java
deleted file mode 100644
index 86349c9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java
+++ /dev/null
@@ -1,238 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-import java.io.IOException;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@code BlockBasedSource} is a {@link FileBasedSource} where a file consists of blocks of
- * records.
- *
- * <p>{@code BlockBasedSource} should be derived from when a file format does not support efficient
- * seeking to a record in the file, but can support efficient seeking to a block. Alternatively,
- * records in the file cannot be offset-addressed, but blocks can (it is not possible to say
- * that record {code i} starts at offset {@code m}, but it is possible to say that block {@code j}
- *  starts at offset {@code n}).
- *
- * <p>The records that will be read from a {@code BlockBasedSource} that corresponds to a subrange
- * of a file {@code [startOffset, endOffset)} are those records such that the record is contained in
- * a block that starts at offset {@code i}, where {@code i >= startOffset} and
- * {@code i < endOffset}. In other words, a record will be read from the source if its first byte is
- * contained in a block that begins within the range described by the source.
- *
- * <p>This entails that it is possible to determine the start offsets of all blocks in a file.
- *
- * <p>Progress reporting for reading from a {@code BlockBasedSource} is inaccurate. A {@link
- * BlockBasedReader} reports its current offset as {@code (offset of current block) + (current block
- * size) * (fraction of block consumed)}. However, only the offset of the current block is required
- * to be accurately reported by subclass implementations. As such, in the worst case, the current
- * offset is only updated at block boundaries.
- *
- * <p>{@code BlockBasedSource} supports dynamic splitting. However, because records in a {@code
- * BlockBasedSource} are not required to have offsets and progress reporting is inaccurate, {@code
- * BlockBasedReader} only supports splitting at block boundaries.
- * In other words, {@link BlockBasedReader#atSplitPoint} returns true iff the current record is the
- * first record in a block. See {@link FileBasedSource.FileBasedReader} for discussion about split
- * points.
- *
- * @param <T> The type of records to be read from the source.
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public abstract class BlockBasedSource<T> extends FileBasedSource<T> {
-  /**
-   * Creates a {@code BlockBasedSource} based on a file name or pattern. Subclasses must call this
-   * constructor when creating a {@code BlockBasedSource} for a file pattern. See
-   * {@link FileBasedSource} for more information.
-   */
-  public BlockBasedSource(String fileOrPatternSpec, long minBundleSize) {
-    super(fileOrPatternSpec, minBundleSize);
-  }
-
-  /**
-   * Creates a {@code BlockBasedSource} for a single file. Subclasses must call this constructor
-   * when implementing {@link BlockBasedSource#createForSubrangeOfFile}. See documentation in
-   * {@link FileBasedSource}.
-   */
-  public BlockBasedSource(String fileName, long minBundleSize, long startOffset, long endOffset) {
-    super(fileName, minBundleSize, startOffset, endOffset);
-  }
-
-  /**
-   * Creates a {@code BlockBasedSource} for the specified range in a single file.
-   */
-  @Override
-  protected abstract BlockBasedSource<T> createForSubrangeOfFile(
-      String fileName, long start, long end);
-
-  /**
-   * Creates a {@code BlockBasedReader}.
-   */
-  @Override
-  protected abstract BlockBasedReader<T> createSingleFileReader(PipelineOptions options);
-
-  /**
-   * A {@code Block} represents a block of records that can be read.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  protected abstract static class Block<T> {
-    /**
-     * Returns the current record.
-     */
-    public abstract T getCurrentRecord();
-
-    /**
-     * Reads the next record from the block and returns true iff one exists.
-     */
-    public abstract boolean readNextRecord() throws IOException;
-
-    /**
-     * Returns the fraction of the block already consumed, if possible, as a value in
-     * {@code [0, 1]}. It should not include the current record. Successive results from this method
-     * must be monotonically increasing.
-     *
-     * <p>If it is not possible to compute the fraction of the block consumed this method may
-     * return zero. For example, when the total number of records in the block is unknown.
-     */
-    public abstract double getFractionOfBlockConsumed();
-  }
-
-  /**
-   * A {@code Reader} that reads records from a {@link BlockBasedSource}. If the source is a
-   * subrange of a file, the blocks that will be read by this reader are those such that the first
-   * byte of the block is within the range {@code [start, end)}.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  protected abstract static class BlockBasedReader<T> extends FileBasedReader<T> {
-    private boolean atSplitPoint;
-
-    protected BlockBasedReader(BlockBasedSource<T> source) {
-      super(source);
-    }
-
-    /**
-     * Read the next block from the input.
-     */
-    public abstract boolean readNextBlock() throws IOException;
-
-    /**
-     * Returns the current block (the block that was read by the last successful call to
-     * {@link BlockBasedReader#readNextBlock}). May return null initially, or if no block has been
-     * successfully read.
-     */
-    @Nullable
-    public abstract Block<T> getCurrentBlock();
-
-    /**
-     * Returns the size of the current block in bytes as it is represented in the underlying file,
-     * if possible. This method may return {@code 0} if the size of the current block is unknown.
-     *
-     * <p>The size returned by this method must be such that for two successive blocks A and B,
-     * {@code offset(A) + size(A) <= offset(B)}. If this is not satisfied, the progress reported
-     * by the {@code BlockBasedReader} will be non-monotonic and will interfere with the quality
-     * (but not correctness) of dynamic work rebalancing.
-     *
-     * <p>This method and {@link Block#getFractionOfBlockConsumed} are used to provide an estimate
-     * of progress within a block ({@code getCurrentBlock().getFractionOfBlockConsumed() *
-     * getCurrentBlockSize()}). It is acceptable for the result of this computation to be {@code 0},
-     * but progress estimation will be inaccurate.
-     */
-    public abstract long getCurrentBlockSize();
-
-    /**
-     * Returns the largest offset such that starting to read from that offset includes the current
-     * block.
-     */
-    public abstract long getCurrentBlockOffset();
-
-    @Override
-    public final T getCurrent() throws NoSuchElementException {
-      Block<T> currentBlock = getCurrentBlock();
-      if (currentBlock == null) {
-        throw new NoSuchElementException(
-            "No block has been successfully read from " + getCurrentSource());
-      }
-      return currentBlock.getCurrentRecord();
-    }
-
-    /**
-     * Returns true if the reader is at a split point. A {@code BlockBasedReader} is at a split
-     * point if the current record is the first record in a block. In other words, split points
-     * are block boundaries.
-     */
-    @Override
-    protected boolean isAtSplitPoint() {
-      return atSplitPoint;
-    }
-
-    /**
-     * Reads the next record from the {@link #getCurrentBlock() current block} if
-     * possible. Will call {@link #readNextBlock()} to advance to the next block if not.
-     *
-     * <p>The first record read from a block is treated as a split point.
-     */
-    @Override
-    protected final boolean readNextRecord() throws IOException {
-      atSplitPoint = false;
-
-      while (getCurrentBlock() == null || !getCurrentBlock().readNextRecord()) {
-        if (!readNextBlock()) {
-          return false;
-        }
-        // The first record in a block is a split point.
-        atSplitPoint = true;
-      }
-      return true;
-    }
-
-    @Override
-    public Double getFractionConsumed() {
-      if (getCurrentSource().getEndOffset() == Long.MAX_VALUE) {
-        return null;
-      }
-      Block<T> currentBlock = getCurrentBlock();
-      if (currentBlock == null) {
-        // There is no current block (i.e., the read has not yet begun).
-        return 0.0;
-      }
-      long currentBlockOffset = getCurrentBlockOffset();
-      long startOffset = getCurrentSource().getStartOffset();
-      long endOffset = getCurrentSource().getEndOffset();
-      double fractionAtBlockStart =
-          ((double) (currentBlockOffset - startOffset)) / (endOffset - startOffset);
-      double fractionAtBlockEnd =
-          ((double) (currentBlockOffset + getCurrentBlockSize() - startOffset)
-              / (endOffset - startOffset));
-      return Math.min(
-          1.0,
-          fractionAtBlockStart
-          + currentBlock.getFractionOfBlockConsumed()
-            * (fractionAtBlockEnd - fractionAtBlockStart));
-    }
-
-    @Override
-    protected long getCurrentOffset() {
-      return getCurrentBlockOffset();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java
deleted file mode 100644
index 2bad6cf..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java
+++ /dev/null
@@ -1,272 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
-
-import com.google.api.client.util.BackOff;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.IntervalBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.ValueWithRecordId;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.util.concurrent.Uninterruptibles;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.TimeUnit;
-
-
-/**
- * {@link PTransform} that reads a bounded amount of data from an {@link UnboundedSource},
- * specified as one or both of a maximum number of elements or a maximum period of time to read.
- *
- * <p>Created by {@link Read}.
- */
-class BoundedReadFromUnboundedSource<T> extends PTransform<PInput, PCollection<T>> {
-  private final UnboundedSource<T, ?> source;
-  private final long maxNumRecords;
-  private final Duration maxReadTime;
-
-  /**
-   * 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.
-   *
-   * <p>This may take a long time to execute if the splits of this source are slow to read
-   * records.
-   */
-  public BoundedReadFromUnboundedSource<T> withMaxNumRecords(long maxNumRecords) {
-    return new BoundedReadFromUnboundedSource<T>(source, maxNumRecords, maxReadTime);
-  }
-
-  /**
-   * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount
-   * of data from the given {@link UnboundedSource}.  The bound is specified as an amount
-   * of time to read for.  Each split of the source will read for this much time.
-   */
-  public BoundedReadFromUnboundedSource<T> withMaxReadTime(Duration maxReadTime) {
-    return new BoundedReadFromUnboundedSource<T>(source, maxNumRecords, maxReadTime);
-  }
-
-  BoundedReadFromUnboundedSource(
-      UnboundedSource<T, ?> source, long maxNumRecords, Duration maxReadTime) {
-    this.source = source;
-    this.maxNumRecords = maxNumRecords;
-    this.maxReadTime = maxReadTime;
-  }
-
-  @Override
-  public PCollection<T> apply(PInput input) {
-    PCollection<ValueWithRecordId<T>> read = Pipeline.applyTransform(input,
-        Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime)));
-    if (source.requiresDeduping()) {
-      read = read.apply(RemoveDuplicates.withRepresentativeValueFn(
-          new SerializableFunction<ValueWithRecordId<T>, byte[]>() {
-            @Override
-            public byte[] apply(ValueWithRecordId<T> input) {
-              return input.getId();
-            }
-          }));
-    }
-    return read.apply(ValueWithRecordId.<T>stripIds());
-  }
-
-  @Override
-  protected Coder<T> getDefaultOutputCoder() {
-    return source.getDefaultOutputCoder();
-  }
-
-  @Override
-  public String getKindString() {
-    return "Read(" + approximateSimpleName(source.getClass()) + ")";
-  }
-
-  private static class UnboundedToBoundedSourceAdapter<T>
-      extends BoundedSource<ValueWithRecordId<T>> {
-    private final UnboundedSource<T, ?> source;
-    private final long maxNumRecords;
-    private final Duration maxReadTime;
-
-    private UnboundedToBoundedSourceAdapter(
-        UnboundedSource<T, ?> source, long maxNumRecords, Duration maxReadTime) {
-      this.source = source;
-      this.maxNumRecords = maxNumRecords;
-      this.maxReadTime = maxReadTime;
-    }
-
-    /**
-     * Divide the given number of records into {@code numSplits} approximately
-     * equal parts that sum to {@code numRecords}.
-     */
-    private static long[] splitNumRecords(long numRecords, int numSplits) {
-      long[] splitNumRecords = new long[numSplits];
-      for (int i = 0; i < numSplits; i++) {
-        splitNumRecords[i] = numRecords / numSplits;
-      }
-      for (int i = 0; i < numRecords % numSplits; i++) {
-        splitNumRecords[i] = splitNumRecords[i] + 1;
-      }
-      return splitNumRecords;
-    }
-
-    /**
-     * Pick a number of initial splits based on the number of records expected to be processed.
-     */
-    private static int numInitialSplits(long numRecords) {
-      final int maxSplits = 100;
-      final long recordsPerSplit = 10000;
-      return (int) Math.min(maxSplits, numRecords / recordsPerSplit + 1);
-    }
-
-    @Override
-    public List<? extends BoundedSource<ValueWithRecordId<T>>> splitIntoBundles(
-        long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
-      List<UnboundedToBoundedSourceAdapter<T>> result = new ArrayList<>();
-      int numInitialSplits = numInitialSplits(maxNumRecords);
-      List<? extends UnboundedSource<T, ?>> splits =
-          source.generateInitialSplits(numInitialSplits, options);
-      int numSplits = splits.size();
-      long[] numRecords = splitNumRecords(maxNumRecords, numSplits);
-      for (int i = 0; i < numSplits; i++) {
-        result.add(
-            new UnboundedToBoundedSourceAdapter<T>(splits.get(i), numRecords[i], maxReadTime));
-      }
-      return result;
-    }
-
-    @Override
-    public long getEstimatedSizeBytes(PipelineOptions options) {
-      // No way to estimate bytes, so returning 0.
-      return 0L;
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) {
-      return false;
-    }
-
-    @Override
-    public Coder<ValueWithRecordId<T>> getDefaultOutputCoder() {
-      return ValueWithRecordId.ValueWithRecordIdCoder.of(source.getDefaultOutputCoder());
-    }
-
-    @Override
-    public void validate() {
-      source.validate();
-    }
-
-    @Override
-    public BoundedReader<ValueWithRecordId<T>> createReader(PipelineOptions options) {
-      return new Reader(source.createReader(options, null));
-    }
-
-    private class Reader extends BoundedReader<ValueWithRecordId<T>> {
-      private long recordsRead = 0L;
-      private Instant endTime = Instant.now().plus(maxReadTime);
-      private UnboundedSource.UnboundedReader<T> reader;
-
-      private Reader(UnboundedSource.UnboundedReader<T> reader) {
-        this.recordsRead = 0L;
-        if (maxReadTime != null) {
-          this.endTime = Instant.now().plus(maxReadTime);
-        } else {
-          this.endTime = null;
-        }
-        this.reader = reader;
-      }
-
-      @Override
-      public boolean start() throws IOException {
-        if (maxNumRecords <= 0 || (maxReadTime != null && maxReadTime.getMillis() == 0)) {
-          return false;
-        }
-
-        recordsRead++;
-        if (reader.start()) {
-          return true;
-        } else {
-          return advanceWithBackoff();
-        }
-      }
-
-      @Override
-      public boolean advance() throws IOException {
-        if (recordsRead >= maxNumRecords) {
-          finalizeCheckpoint();
-          return false;
-        }
-        recordsRead++;
-        return advanceWithBackoff();
-      }
-
-      private boolean advanceWithBackoff() throws IOException {
-        // Try reading from the source with exponential backoff
-        BackOff backoff = new IntervalBoundedExponentialBackOff(10000, 10);
-        long nextSleep = backoff.nextBackOffMillis();
-        while (nextSleep != BackOff.STOP) {
-          if (endTime != null && Instant.now().isAfter(endTime)) {
-            finalizeCheckpoint();
-            return false;
-          }
-          if (reader.advance()) {
-            return true;
-          }
-          Uninterruptibles.sleepUninterruptibly(nextSleep, TimeUnit.MILLISECONDS);
-          nextSleep = backoff.nextBackOffMillis();
-        }
-        finalizeCheckpoint();
-        return false;
-      }
-
-      private void finalizeCheckpoint() throws IOException {
-        reader.getCheckpointMark().finalizeCheckpoint();
-      }
-
-      @Override
-      public ValueWithRecordId<T> getCurrent() throws NoSuchElementException {
-        return new ValueWithRecordId<>(reader.getCurrent(), reader.getCurrentRecordId());
-      }
-
-      @Override
-      public Instant getCurrentTimestamp() throws NoSuchElementException {
-        return reader.getCurrentTimestamp();
-      }
-
-      @Override
-      public void close() throws IOException {
-        reader.close();
-      }
-
-      @Override
-      public BoundedSource<ValueWithRecordId<T>> getCurrentSource() {
-        return UnboundedToBoundedSourceAdapter.this;
-      }
-    }
-  }
-}



[53/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 39a33cc..359ad23 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
@@ -23,20 +23,20 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.TestPipeline;
+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.ParDo;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TimestampedValue;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 1df0d96..5cbf044 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
@@ -15,25 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+package org.apache.beam.sdk.transforms.windowing;
+
+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.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.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.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.TimestampedValue;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
index a8f2c1e..b07167c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -46,12 +46,12 @@ public class ApiSurfaceTest {
   @Test
   public void testOurApiSurface() throws Exception {
     ApiSurface checkedApiSurface = ApiSurface.getSdkApiSurface()
-      .pruningClassName("com.google.cloud.dataflow.sdk.runners.worker.StateFetcher")
-      .pruningClassName("com.google.cloud.dataflow.sdk.util.common.ReflectHelpers")
-      .pruningClassName("com.google.cloud.dataflow.sdk.DataflowMatchers")
-      .pruningClassName("com.google.cloud.dataflow.sdk.TestUtils")
-      .pruningClassName("com.google.cloud.dataflow.sdk.WindowMatchers")
-      .pruningClassName("com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers");
+      .pruningClassName("org.apache.beam.sdk.runners.worker.StateFetcher")
+      .pruningClassName("org.apache.beam.sdk.util.common.ReflectHelpers")
+      .pruningClassName("org.apache.beam.sdk.DataflowMatchers")
+      .pruningClassName("org.apache.beam.sdk.TestUtils")
+      .pruningClassName("org.apache.beam.sdk.WindowMatchers")
+      .pruningClassName("org.apache.beam.sdk.transforms.display.DisplayDataMatchers");
 
     checkedApiSurface.getExposedClasses();
 
@@ -76,7 +76,7 @@ public class ApiSurfaceTest {
   }
 
   private boolean classIsAllowed(Class<?> clazz) {
-    return  clazz.getName().startsWith("com.google.cloud.dataflow");
+    return  clazz.getName().startsWith("org.apache.beam");
   }
 
   //////////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
index 94a8e88..08f5f56 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
@@ -25,8 +25,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+
 import com.google.api.client.util.BackOff;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
index fa1b59b..b4a075c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-
 import com.google.api.client.util.BackOff;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
index f186710..38e921a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+import org.apache.beam.sdk.util.AvroUtils.AvroMetadata;
+
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.util.AvroUtils.AvroMetadata;
 import com.google.common.collect.Lists;
 
 import org.apache.avro.Schema;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
index 197b7e8..3e1528f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaceForTest;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaceForTest;
 
 import org.joda.time.Instant;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
index 866f6e3..74a2da2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java
@@ -15,13 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+
 import com.google.api.client.googleapis.json.GoogleJsonError;
 import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
 import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
@@ -39,8 +42,6 @@ import com.google.api.services.bigquery.model.Job;
 import com.google.api.services.bigquery.model.JobConfigurationLoad;
 import com.google.api.services.bigquery.model.JobStatus;
 import com.google.api.services.bigquery.model.TableReference;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ea29e5d..7d9c8a8 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Verify.verifyNotNull;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
@@ -27,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.testing.ExpectedLogs;
+
 import com.google.api.client.googleapis.json.GoogleJsonError;
 import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
 import com.google.api.client.googleapis.json.GoogleJsonErrorContainer;
@@ -44,7 +47,6 @@ import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
 import com.google.cloud.hadoop.util.RetryBoundedBackOff;
 import com.google.common.collect.ImmutableList;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
index 24750fe..e92f2c6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableRowIteratorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7cb2c46..65fbeb7 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
@@ -30,6 +30,11 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Sum;
+
 import com.google.api.services.bigquery.Bigquery;
 import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableCell;
@@ -40,10 +45,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.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java
index ef48fcf..09d8992 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder.Context;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.collection.IsIterableContainingInOrder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
index 0ecdb7b..0733c4c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.util.CoderUtils.makeCloudEncoding;
 
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.testing.CoderPropertiesTest.ClosingCoder;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.testing.CoderPropertiesTest.ClosingCoder;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
index 173c1fa..fdb69e4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.withSettings;
 
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.state.StateContexts;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java
index 9525984..3f96cf2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java
@@ -15,29 +15,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM;
 
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MIN;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.withSettings;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.IterableCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterProvider;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.IterableCombineFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterProvider;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
+
 import com.google.common.collect.Iterables;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
index c975e08..fdcd1bd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertSame;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
 
 import org.joda.time.Instant;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java
index 87b1012..d717caf 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java
index cfd05fa..9819a9b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java
index 0d86095..b510408 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
index 2d8ba56..7f74620 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.theInstance;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
index bc2a514..a66f74f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterAll;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterFirst;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.AfterAll;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
 
 /**
  * Generalized tests for {@link FinishedTriggers} implementations.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
index a84b73e..b3b1856 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.theInstance;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
index 8c1a918..aae3137 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
index 70fbe1a..49c7bc4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.contains;
 import static org.junit.Assert.assertEquals;
@@ -27,6 +27,11 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
 import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.http.HttpRequest;
@@ -47,10 +52,6 @@ import com.google.api.services.storage.Storage;
 import com.google.api.services.storage.model.Bucket;
 import com.google.api.services.storage.model.Objects;
 import com.google.api.services.storage.model.StorageObject;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
 import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
 import com.google.cloud.hadoop.util.ClientRequestHelper;
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
index 6c399a8..d21edd1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
@@ -15,28 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.TestUtils.KvMatcher;
-import com.google.cloud.dataflow.sdk.WindowMatchers;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.TestUtils.KvMatcher;
+import org.apache.beam.sdk.WindowMatchers;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
index ae40d74..d9c786d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java
index 5768dec..c79f1a1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java
index d5941e3..8d197ba 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.hamcrest.Matchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
index e3f8fd2..cde0958 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOffTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java
index 509fa23..2151f88 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespaces;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
index f042ba2..d929d39 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java
index 6783b81..7ab8322 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateInternals;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
index b367c7e..0763912 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
 
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java
index 568d437..fe2014a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RandomAccessDataTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RandomAccessDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RandomAccessDataTest.java
index c54108c..b990212 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RandomAccessDataTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RandomAccessDataTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -23,10 +23,11 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.RandomAccessData.RandomAccessDataCoder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.RandomAccessData.RandomAccessDataCoder;
+
 import com.google.common.primitives.UnsignedBytes;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
index ef499f2..1939b20 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
 
-import static com.google.cloud.dataflow.sdk.WindowMatchers.isSingleWindowedValue;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -26,38 +27,39 @@ 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.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.withSettings;
 
-import com.google.cloud.dataflow.sdk.WindowMatchers;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterFirst;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.WindowMatchers;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+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 org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
index 86c2d11..f0d2a44 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
@@ -15,45 +15,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.TriggerBuilder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.util.state.State;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.WatermarkHoldState;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+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.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.Sum;
+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.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
index 7fafe0c..f4ae996 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+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.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
index 883ea4c..cabde7f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 789badf..7d212d4 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
index a99a206..d9e7593 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.DeterministicStandardCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.DeterministicStandardCoder;
+
 import com.google.common.collect.ImmutableList;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializerTest.java
index f396c08..6c5d0bd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializerTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDouble;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDouble;
+import static org.apache.beam.sdk.util.Structs.addLong;
+import static org.apache.beam.sdk.util.Structs.addString;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
index 7405808..680422b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.is;
 import static org.mockito.Mockito.mock;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.util.BaseExecutionContext.StepContext;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
index 29f7bdc..60eeb0d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 27e0a69..408af83 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
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PDone;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java
index 9e1b4df..59dbb56 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java
@@ -15,24 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDouble;
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLongs;
-import static com.google.cloud.dataflow.sdk.util.Structs.addNull;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
-import static com.google.cloud.dataflow.sdk.util.Structs.getBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.getInt;
-import static com.google.cloud.dataflow.sdk.util.Structs.getListOfMaps;
-import static com.google.cloud.dataflow.sdk.util.Structs.getLong;
-import static com.google.cloud.dataflow.sdk.util.Structs.getObject;
-import static com.google.cloud.dataflow.sdk.util.Structs.getString;
-import static com.google.cloud.dataflow.sdk.util.Structs.getStrings;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDouble;
+import static org.apache.beam.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addLong;
+import static org.apache.beam.sdk.util.Structs.addLongs;
+import static org.apache.beam.sdk.util.Structs.addNull;
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addStringList;
+import static org.apache.beam.sdk.util.Structs.getBoolean;
+import static org.apache.beam.sdk.util.Structs.getDictionary;
+import static org.apache.beam.sdk.util.Structs.getInt;
+import static org.apache.beam.sdk.util.Structs.getListOfMaps;
+import static org.apache.beam.sdk.util.Structs.getLong;
+import static org.apache.beam.sdk.util.Structs.getObject;
+import static org.apache.beam.sdk.util.Structs.getString;
+import static org.apache.beam.sdk.util.Structs.getStrings;
 
 import org.hamcrest.Matchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimeUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimeUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimeUtilTest.java
index e376cdf..b318dee 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimeUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimeUtilTest.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudDuration;
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
+import static org.apache.beam.sdk.util.TimeUtil.toCloudDuration;
+import static org.apache.beam.sdk.util.TimeUtil.toCloudTime;
 
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration;
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration;
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
index 7256681..baf911a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerDataCoder;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder;
+import org.apache.beam.sdk.util.state.StateNamespaces;
 
 import org.joda.time.Instant;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 61eeb90..10d3d35 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
@@ -15,32 +15,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 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.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.TriggerBuilder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.ActiveWindowSet.MergeCallback;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.util.state.State;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces.WindowAndTriggerNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces.WindowNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.WatermarkHoldState;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+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;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowAndTriggerNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+import org.apache.beam.sdk.util.state.StateTag;
+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.base.Throwables;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
index 902d796..dac147c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
index 2f8239f..30761e7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java
index b855055..5728977 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.testing.ExpectedLogs;
 
 import com.google.api.client.http.GenericUrl;
 import com.google.api.client.http.HttpResponse;
-
 import com.google.api.client.testing.http.HttpTesting;
 import com.google.api.client.testing.http.MockHttpTransport;
 import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
 
 import org.junit.Rule;
 import org.junit.Test;



[72/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
Initial update of imports

Performed using the following script

    find . -iname "*.java" -exec sed -i -e \
        "s/package com.google.cloud.dataflow/package org.apache.beam/" {} +

fixup! Rewrite import statements

Performed using the following command:

    find . -iname "*.java" -exec sed -i -e \
        "s/com.google.cloud.dataflow/org.apache.beam/" {} +

Revert changes to imports of proto2 coders (deployed to maven)

fixup! Update mentions of RunnableOnService

fixup! Organize imports using Eclipse

fixup! move sdk.properties

fixup! update package generated protos will be in

fixup! Move the PipelineOptionsFactoryJava8Test

fixup! Update user.avsc

fixup! Update runner and core pom.xml

fixup! Update dependencies in contrib pom.xmls

fixup! Update ApiSurface

fixup! Fix class names in test strings.

fixup! Update travis/test_wordcount.sh

fixup! Organize imports.

fixup! javadoc beam


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

Branch: refs/heads/master
Commit: b972445412d4c7c563e57a2260e99b9b5e3c7a32
Parents: 0393a79
Author: bchambers <bc...@google.com>
Authored: Wed Apr 13 14:39:02 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Wed Apr 13 21:23:06 2016 -0700

----------------------------------------------------------------------
 contrib/hadoop/pom.xml                          |   8 +-
 .../apache/contrib/hadoop/HadoopFileSource.java |  26 +--
 .../apache/contrib/hadoop/WritableCoder.java    |  12 +-
 .../contrib/hadoop/HadoopFileSourceTest.java    |  16 +-
 .../contrib/hadoop/WritableCoderTest.java       |   4 +-
 contrib/join-library/pom.xml                    |   8 +-
 .../org/apache/contrib/joinlibrary/Join.java    |  22 +-
 .../contrib/joinlibrary/InnerJoinTest.java      |  16 +-
 .../contrib/joinlibrary/OuterLeftJoinTest.java  |  16 +-
 .../contrib/joinlibrary/OuterRightJoinTest.java |  16 +-
 examples/java/pom.xml                           |   4 +-
 .../beam/examples/DebuggingWordCount.java       |  34 +--
 .../apache/beam/examples/MinimalWordCount.java  |  24 +-
 .../apache/beam/examples/WindowedWordCount.java |  39 ++--
 .../org/apache/beam/examples/WordCount.java     |  42 ++--
 .../examples/common/DataflowExampleOptions.java |   8 +-
 .../examples/common/DataflowExampleUtils.java   |  33 +--
 .../common/ExampleBigQueryTableOptions.java     |  13 +-
 ...xamplePubsubTopicAndSubscriptionOptions.java |  12 +-
 .../common/ExamplePubsubTopicOptions.java       |  12 +-
 .../examples/common/PubsubFileInjector.java     |  23 +-
 .../beam/examples/complete/AutoComplete.java    |  69 +++---
 .../examples/complete/StreamingWordExtract.java |  29 +--
 .../apache/beam/examples/complete/TfIdf.java    |  72 +++---
 .../examples/complete/TopWikipediaSessions.java |  47 ++--
 .../examples/complete/TrafficMaxLaneFlow.java   |  51 +++--
 .../beam/examples/complete/TrafficRoutes.java   |  49 ++--
 .../examples/cookbook/BigQueryTornadoes.java    |  29 +--
 .../cookbook/CombinePerKeyExamples.java         |  35 +--
 .../examples/cookbook/DatastoreWordCount.java   |  29 +--
 .../beam/examples/cookbook/DeDupExample.java    |  26 +--
 .../beam/examples/cookbook/FilterExamples.java  |  31 +--
 .../beam/examples/cookbook/JoinExamples.java    |  33 +--
 .../examples/cookbook/MaxPerKeyExamples.java    |  29 +--
 .../beam/examples/cookbook/TriggerExample.java  |  68 +++---
 .../beam/examples/DebuggingWordCountTest.java   |   2 +-
 .../org/apache/beam/examples/WordCountTest.java |  26 +--
 .../examples/complete/AutoCompleteTest.java     |  36 +--
 .../beam/examples/complete/TfIdfTest.java       |  22 +-
 .../complete/TopWikipediaSessionsTest.java      |  15 +-
 .../cookbook/BigQueryTornadoesTest.java         |  11 +-
 .../cookbook/CombinePerKeyExamplesTest.java     |  11 +-
 .../examples/cookbook/DeDupExampleTest.java     |  18 +-
 .../examples/cookbook/FilterExamplesTest.java   |   9 +-
 .../examples/cookbook/JoinExamplesTest.java     |  23 +-
 .../cookbook/MaxPerKeyExamplesTest.java         |  11 +-
 .../examples/cookbook/TriggerExampleTest.java   |  33 +--
 .../beam/examples/MinimalWordCountJava8.java    |  24 +-
 .../beam/examples/complete/game/GameStats.java  |  64 +++---
 .../examples/complete/game/HourlyTeamScore.java |  31 ++-
 .../examples/complete/game/LeaderBoard.java     |  48 ++--
 .../beam/examples/complete/game/UserScore.java  |  42 ++--
 .../complete/game/injector/Injector.java        |   2 +-
 .../complete/game/injector/InjectorUtils.java   |   3 +-
 .../injector/RetryHttpInitializerWrapper.java   |   2 +-
 .../complete/game/utils/WriteToBigQuery.java    |  29 +--
 .../game/utils/WriteWindowedToBigQuery.java     |  21 +-
 .../examples/MinimalWordCountJava8Test.java     |  27 +--
 .../examples/complete/game/GameStatsTest.java   |  18 +-
 .../complete/game/HourlyTeamScoreTest.java      |  32 +--
 .../examples/complete/game/UserScoreTest.java   |  34 +--
 .../beam/runners/flink/examples/TFIDF.java      |  68 +++---
 .../beam/runners/flink/examples/WordCount.java  |  18 +-
 .../flink/examples/streaming/AutoComplete.java  |  28 +--
 .../flink/examples/streaming/JoinExamples.java  |  30 +--
 .../KafkaWindowedWordCountExample.java          |  20 +-
 .../examples/streaming/WindowedWordCount.java   |  18 +-
 .../FlinkPipelineExecutionEnvironment.java      |   8 +-
 .../runners/flink/FlinkPipelineOptions.java     |  13 +-
 .../beam/runners/flink/FlinkPipelineRunner.java |  23 +-
 .../runners/flink/FlinkRunnerRegistrar.java     |  11 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |  12 +-
 .../apache/beam/runners/flink/io/ConsoleIO.java |   8 +-
 .../FlinkBatchPipelineTranslator.java           |  15 +-
 .../FlinkBatchTransformTranslators.java         |  58 ++---
 .../FlinkBatchTranslationContext.java           |  21 +-
 .../translation/FlinkPipelineTranslator.java    |   4 +-
 .../FlinkStreamingPipelineTranslator.java       |  16 +-
 .../FlinkStreamingTransformTranslators.java     |  54 +++--
 .../FlinkStreamingTranslationContext.java       |  14 +-
 .../FlinkCoGroupKeyedListAggregator.java        |  11 +-
 .../functions/FlinkCreateFunction.java          |   5 +-
 .../functions/FlinkDoFnFunction.java            |  35 +--
 .../FlinkKeyedListAggregationFunction.java      |   7 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |  33 +--
 .../FlinkMultiOutputPruningFunction.java        |   3 +-
 .../functions/FlinkPartialReduceFunction.java   |   9 +-
 .../functions/FlinkReduceFunction.java          |  10 +-
 .../flink/translation/functions/UnionCoder.java |  15 +-
 .../translation/types/CoderComparator.java      |   5 +-
 .../translation/types/CoderTypeInformation.java |  10 +-
 .../translation/types/CoderTypeSerializer.java  |   7 +-
 .../translation/types/KvCoderComperator.java    |   9 +-
 .../types/KvCoderTypeInformation.java           |  10 +-
 .../types/VoidCoderTypeSerializer.java          |   2 +-
 .../wrappers/CombineFnAggregatorWrapper.java    |  10 +-
 .../wrappers/DataInputViewWrapper.java          |   2 +-
 .../wrappers/DataOutputViewWrapper.java         |   2 +-
 .../SerializableFnAggregatorWrapper.java        |  10 +-
 .../translation/wrappers/SinkOutputFormat.java  |  11 +-
 .../translation/wrappers/SourceInputFormat.java |  11 +-
 .../translation/wrappers/SourceInputSplit.java  |   3 +-
 .../streaming/FlinkAbstractParDoWrapper.java    |  30 +--
 .../FlinkGroupAlsoByWindowWrapper.java          |  75 ++++--
 .../streaming/FlinkGroupByKeyWrapper.java       |  11 +-
 .../streaming/FlinkParDoBoundMultiWrapper.java  |  18 +-
 .../streaming/FlinkParDoBoundWrapper.java       |  26 ++-
 .../io/FlinkStreamingCreateFunction.java        |  10 +-
 .../streaming/io/UnboundedFlinkSource.java      |  12 +-
 .../streaming/io/UnboundedSocketSource.java     |  14 +-
 .../streaming/io/UnboundedSourceWrapper.java    |  18 +-
 .../state/AbstractFlinkTimerInternals.java      |  17 +-
 .../streaming/state/FlinkStateInternals.java    |  35 ++-
 .../streaming/state/StateCheckpointReader.java  |   2 +
 .../streaming/state/StateCheckpointUtils.java   |  16 +-
 .../streaming/state/StateCheckpointWriter.java  |   2 +
 .../apache/beam/runners/flink/AvroITCase.java   |  16 +-
 .../beam/runners/flink/FlattenizeITCase.java    |  14 +-
 .../runners/flink/FlinkRunnerRegistrarTest.java |   9 +-
 .../beam/runners/flink/FlinkTestPipeline.java   |  16 +-
 .../beam/runners/flink/JoinExamplesITCase.java  |  11 +-
 .../runners/flink/MaybeEmptyTestITCase.java     |  13 +-
 .../runners/flink/ParDoMultiOutputITCase.java   |  20 +-
 .../beam/runners/flink/ReadSourceITCase.java    |  22 +-
 .../flink/RemoveDuplicatesEmptyITCase.java      |  14 +-
 .../runners/flink/RemoveDuplicatesITCase.java   |  14 +-
 .../beam/runners/flink/SideInputITCase.java     |  15 +-
 .../apache/beam/runners/flink/TfIdfITCase.java  |  20 +-
 .../beam/runners/flink/WordCountITCase.java     |  16 +-
 .../runners/flink/WordCountJoin2ITCase.java     |  26 ++-
 .../runners/flink/WordCountJoin3ITCase.java     |  26 ++-
 .../beam/runners/flink/WriteSinkITCase.java     |  20 +-
 .../flink/streaming/GroupAlsoByWindowTest.java  |  37 +--
 .../flink/streaming/GroupByNullKeyTest.java     |  25 +-
 .../flink/streaming/StateSerializationTest.java |  41 ++--
 .../streaming/TopWikipediaSessionsITCase.java   |  22 +-
 .../flink/streaming/UnboundedSourceITCase.java  |  44 ++--
 .../beam/runners/flink/util/JoinExamples.java   |  33 +--
 runners/google-cloud-dataflow-java/pom.xml      |  12 +-
 .../BlockingDataflowPipelineOptions.java        |   4 +-
 .../beam/sdk/options/CloudDebuggerOptions.java  |   2 +-
 .../options/DataflowPipelineDebugOptions.java   |  14 +-
 .../sdk/options/DataflowPipelineOptions.java    |   5 +-
 .../DataflowPipelineWorkerPoolOptions.java      |   4 +-
 .../sdk/options/DataflowProfilingOptions.java   |   2 +-
 .../options/DataflowWorkerHarnessOptions.java   |   2 +-
 .../options/DataflowWorkerLoggingOptions.java   |   2 +-
 .../runners/BlockingDataflowPipelineRunner.java |  21 +-
 .../DataflowJobAlreadyExistsException.java      |   2 +-
 .../DataflowJobAlreadyUpdatedException.java     |   2 +-
 .../runners/DataflowJobCancelledException.java  |   2 +-
 .../beam/sdk/runners/DataflowJobException.java  |   2 +-
 .../runners/DataflowJobExecutionException.java  |   2 +-
 .../runners/DataflowJobUpdatedException.java    |   2 +-
 .../beam/sdk/runners/DataflowPipeline.java      |  10 +-
 .../beam/sdk/runners/DataflowPipelineJob.java   |  21 +-
 .../sdk/runners/DataflowPipelineRegistrar.java  |  11 +-
 .../sdk/runners/DataflowPipelineRunner.java     | 229 ++++++++++---------
 .../runners/DataflowPipelineRunnerHooks.java    |   2 +-
 .../sdk/runners/DataflowPipelineTranslator.java | 112 ++++-----
 .../sdk/runners/DataflowServiceException.java   |   2 +-
 .../sdk/runners/dataflow/AssignWindows.java     |  18 +-
 .../runners/dataflow/BigQueryIOTranslator.java  |  15 +-
 .../sdk/runners/dataflow/CustomSources.java     |  22 +-
 .../dataflow/DataflowAggregatorTransforms.java  |  11 +-
 .../dataflow/DataflowMetricUpdateExtractor.java |  11 +-
 .../runners/dataflow/PubsubIOTranslator.java    |  14 +-
 .../sdk/runners/dataflow/ReadTranslator.java    |  27 +--
 .../beam/sdk/runners/dataflow/package-info.java |   4 +-
 .../testing/TestDataflowPipelineOptions.java    |   4 +-
 .../sdk/testing/TestDataflowPipelineRunner.java |  27 +--
 .../beam/sdk/util/DataflowPathValidator.java    |   9 +-
 .../apache/beam/sdk/util/DataflowTransport.java |   9 +-
 .../org/apache/beam/sdk/util/GcsStager.java     |   9 +-
 .../apache/beam/sdk/util/MonitoringUtil.java    |   9 +-
 .../org/apache/beam/sdk/util/PackageUtil.java   |   2 +-
 .../java/org/apache/beam/sdk/util/Stager.java   |   2 +-
 .../apache/beam/sdk/io/DataflowTextIOTest.java  |  17 +-
 .../DataflowPipelineDebugOptionsTest.java       |   2 +-
 .../options/DataflowPipelineOptionsTest.java    |   6 +-
 .../options/DataflowProfilingOptionsTest.java   |   2 +-
 .../DataflowWorkerLoggingOptionsTest.java       |   8 +-
 .../BlockingDataflowPipelineRunnerTest.java     |  22 +-
 .../sdk/runners/DataflowPipelineJobTest.java    |  27 +--
 .../runners/DataflowPipelineRegistrarTest.java  |   9 +-
 .../sdk/runners/DataflowPipelineRunnerTest.java | 104 ++++-----
 .../beam/sdk/runners/DataflowPipelineTest.java  |  10 +-
 .../runners/DataflowPipelineTranslatorTest.java |  72 +++---
 .../sdk/runners/dataflow/CustomSourcesTest.java |  40 ++--
 .../testing/TestDataflowPipelineRunnerTest.java |  31 +--
 .../sdk/transforms/DataflowGroupByKeyTest.java  |  32 +--
 .../beam/sdk/transforms/DataflowViewTest.java   |  32 +--
 .../sdk/util/DataflowPathValidatorTest.java     |  10 +-
 .../beam/sdk/util/MonitoringUtilTest.java       |   9 +-
 .../apache/beam/sdk/util/PackageUtilTest.java   |  15 +-
 runners/pom.xml                                 |   2 +-
 .../beam/runners/spark/EvaluationResult.java    |   6 +-
 .../runners/spark/SparkPipelineOptions.java     |  10 +-
 .../beam/runners/spark/SparkPipelineRunner.java |  34 +--
 .../spark/SparkStreamingPipelineOptions.java    |   4 +-
 .../spark/aggregators/NamedAggregators.java     |   7 +-
 .../beam/runners/spark/coders/CoderHelpers.java |   2 +-
 .../runners/spark/coders/NullWritableCoder.java |   2 +-
 .../runners/spark/coders/WritableCoder.java     |   8 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |   6 +-
 .../beam/runners/spark/io/CreateStream.java     |   8 +-
 .../apache/beam/runners/spark/io/KafkaIO.java   |  11 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |  14 +-
 .../spark/io/hadoop/ShardNameBuilder.java       |   4 +-
 .../spark/io/hadoop/ShardNameTemplateAware.java |   2 +-
 .../io/hadoop/ShardNameTemplateHelper.java      |   4 +-
 .../runners/spark/translation/DoFnFunction.java |   6 +-
 .../spark/translation/EvaluationContext.java    |  26 +--
 .../spark/translation/MultiDoFnFunction.java    |   6 +-
 .../translation/SparkPipelineEvaluator.java     |  10 +-
 .../SparkPipelineOptionsFactory.java            |   2 +-
 .../SparkPipelineOptionsRegistrar.java          |   4 +-
 .../SparkPipelineRunnerRegistrar.java           |   4 +-
 .../translation/SparkPipelineTranslator.java    |   2 +-
 .../spark/translation/SparkProcessContext.java  |  28 +--
 .../spark/translation/SparkRuntimeContext.java  |  26 +--
 .../spark/translation/TransformEvaluator.java   |   2 +-
 .../spark/translation/TransformTranslator.java  |  62 +++--
 .../spark/translation/WindowingHelpers.java     |   2 +-
 .../SparkStreamingPipelineOptionsFactory.java   |   2 +-
 .../SparkStreamingPipelineOptionsRegistrar.java |   4 +-
 .../streaming/StreamingEvaluationContext.java   |  18 +-
 .../streaming/StreamingTransformTranslator.java |  42 ++--
 .../StreamingWindowPipelineDetector.java        |  20 +-
 .../runners/spark/util/BroadcastHelper.java     |   2 +-
 .../apache/beam/runners/spark/DeDupTest.java    |  20 +-
 .../beam/runners/spark/EmptyInputTest.java      |  20 +-
 .../beam/runners/spark/SimpleWordCountTest.java |  26 ++-
 .../apache/beam/runners/spark/TfIdfTest.java    |  24 +-
 .../runners/spark/coders/WritableCoderTest.java |   2 +-
 .../beam/runners/spark/io/AvroPipelineTest.java |  17 +-
 .../beam/runners/spark/io/NumShardsTest.java    |  36 +--
 .../io/hadoop/HadoopFileFormatPipelineTest.java |  20 +-
 .../spark/io/hadoop/ShardNameBuilderTest.java   |   4 +-
 .../spark/translation/CombineGloballyTest.java  |  20 +-
 .../spark/translation/CombinePerKeyTest.java    |  28 ++-
 .../spark/translation/DoFnOutputTest.java       |  16 +-
 .../translation/MultiOutputWordCountTest.java   |  34 ++-
 .../spark/translation/SerializationTest.java    |  38 +--
 .../spark/translation/SideEffectsTest.java      |  22 +-
 .../translation/TransformTranslatorTest.java    |  26 +--
 .../translation/WindowedWordCountTest.java      |  21 +-
 .../streaming/FlattenStreamingTest.java         |  30 ++-
 .../streaming/KafkaStreamingTest.java           |  43 ++--
 .../streaming/SimpleStreamingWordCountTest.java |  30 ++-
 .../streaming/utils/EmbeddedKafkaCluster.java   |   7 +-
 .../streaming/utils/PAssertStreaming.java       |   3 +-
 sdks/java/core/pom.xml                          |  16 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  37 +--
 .../org/apache/beam/sdk/PipelineResult.java     |   8 +-
 .../org/apache/beam/sdk/coders/AtomicCoder.java |   2 +-
 .../org/apache/beam/sdk/coders/AvroCoder.java   |  10 +-
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |   2 +-
 .../beam/sdk/coders/BigEndianLongCoder.java     |   2 +-
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |   9 +-
 .../org/apache/beam/sdk/coders/ByteCoder.java   |   2 +-
 .../apache/beam/sdk/coders/ByteStringCoder.java |   5 +-
 .../sdk/coders/CannotProvideCoderException.java |   2 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |  11 +-
 .../apache/beam/sdk/coders/CoderException.java  |   2 +-
 .../apache/beam/sdk/coders/CoderFactories.java  |   4 +-
 .../apache/beam/sdk/coders/CoderFactory.java    |   2 +-
 .../apache/beam/sdk/coders/CoderProvider.java   |   4 +-
 .../apache/beam/sdk/coders/CoderProviders.java  |   7 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |  17 +-
 .../apache/beam/sdk/coders/CollectionCoder.java |   5 +-
 .../org/apache/beam/sdk/coders/CustomCoder.java |  16 +-
 .../apache/beam/sdk/coders/DefaultCoder.java    |   4 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   2 +-
 .../sdk/coders/DeterministicStandardCoder.java  |   2 +-
 .../org/apache/beam/sdk/coders/DoubleCoder.java |   2 +-
 .../apache/beam/sdk/coders/DurationCoder.java   |   4 +-
 .../org/apache/beam/sdk/coders/EntityCoder.java |   2 +-
 .../apache/beam/sdk/coders/InstantCoder.java    |   5 +-
 .../apache/beam/sdk/coders/IterableCoder.java   |   9 +-
 .../beam/sdk/coders/IterableLikeCoder.java      |  11 +-
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |   9 +-
 .../org/apache/beam/sdk/coders/KvCoder.java     |  13 +-
 .../org/apache/beam/sdk/coders/KvCoderBase.java |   4 +-
 .../org/apache/beam/sdk/coders/ListCoder.java   |   5 +-
 .../org/apache/beam/sdk/coders/MapCoder.java    |   7 +-
 .../apache/beam/sdk/coders/MapCoderBase.java    |   4 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |   7 +-
 .../org/apache/beam/sdk/coders/Proto2Coder.java |  11 +-
 .../beam/sdk/coders/SerializableCoder.java      |   6 +-
 .../org/apache/beam/sdk/coders/SetCoder.java    |   5 +-
 .../apache/beam/sdk/coders/StandardCoder.java   |  16 +-
 .../beam/sdk/coders/StringDelegateCoder.java    |   4 +-
 .../apache/beam/sdk/coders/StringUtf8Coder.java |   9 +-
 .../beam/sdk/coders/StructuralByteArray.java    |   2 +-
 .../beam/sdk/coders/TableRowJsonCoder.java      |   2 +-
 .../beam/sdk/coders/TextualIntegerCoder.java    |   2 +-
 .../org/apache/beam/sdk/coders/VarIntCoder.java |   4 +-
 .../apache/beam/sdk/coders/VarLongCoder.java    |   4 +-
 .../org/apache/beam/sdk/coders/VoidCoder.java   |   2 +-
 .../apache/beam/sdk/coders/package-info.java    |  16 +-
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |  23 +-
 .../beam/sdk/coders/protobuf/ProtobufUtil.java  |   5 +-
 .../beam/sdk/coders/protobuf/package-info.java  |   6 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  37 +--
 .../java/org/apache/beam/sdk/io/AvroSource.java |  15 +-
 .../java/org/apache/beam/sdk/io/BigQueryIO.java |  87 +++----
 .../apache/beam/sdk/io/BlockBasedSource.java    |   4 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  25 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |  10 +-
 .../apache/beam/sdk/io/CompressedSource.java    |   7 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |  17 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |  23 +-
 .../org/apache/beam/sdk/io/DatastoreIO.java     |  26 +--
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  43 ++--
 .../org/apache/beam/sdk/io/FileBasedSource.java |  11 +-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   9 +-
 .../org/apache/beam/sdk/io/PubsubClient.java    |   3 +-
 .../apache/beam/sdk/io/PubsubGrpcClient.java    |   6 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |  39 ++--
 .../main/java/org/apache/beam/sdk/io/Read.java  |  26 +--
 .../apache/beam/sdk/io/ShardNameTemplate.java   |   2 +-
 .../main/java/org/apache/beam/sdk/io/Sink.java  |   8 +-
 .../java/org/apache/beam/sdk/io/Source.java     |   4 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  45 ++--
 .../org/apache/beam/sdk/io/UnboundedSource.java |  18 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |  35 ++-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |  15 +-
 .../java/org/apache/beam/sdk/io/XmlSource.java  |  13 +-
 .../apache/beam/sdk/io/bigtable/BigtableIO.java |  44 ++--
 .../beam/sdk/io/bigtable/BigtableService.java   |   7 +-
 .../sdk/io/bigtable/BigtableServiceImpl.java    |   7 +-
 .../beam/sdk/io/bigtable/package-info.java      |   4 +-
 .../org/apache/beam/sdk/io/package-info.java    |   8 +-
 .../org/apache/beam/sdk/io/range/ByteKey.java   |   2 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   2 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   2 +-
 .../beam/sdk/io/range/OffsetRangeTracker.java   |   2 +-
 .../apache/beam/sdk/io/range/RangeTracker.java  |   6 +-
 .../apache/beam/sdk/io/range/package-info.java  |   4 +-
 .../sdk/options/ApplicationNameOptions.java     |   2 +-
 .../beam/sdk/options/BigQueryOptions.java       |   2 +-
 .../org/apache/beam/sdk/options/Default.java    |   2 +-
 .../beam/sdk/options/DefaultValueFactory.java   |   2 +-
 .../apache/beam/sdk/options/Description.java    |   2 +-
 .../beam/sdk/options/DirectPipelineOptions.java |  10 +-
 .../org/apache/beam/sdk/options/GcpOptions.java |   9 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |   7 +-
 .../beam/sdk/options/GoogleApiDebugOptions.java |   2 +-
 .../org/apache/beam/sdk/options/Hidden.java     |   2 +-
 .../beam/sdk/options/PipelineOptions.java       |  19 +-
 .../sdk/options/PipelineOptionsFactory.java     |  15 +-
 .../sdk/options/PipelineOptionsRegistrar.java   |   2 +-
 .../sdk/options/PipelineOptionsValidator.java   |   7 +-
 .../sdk/options/ProxyInvocationHandler.java     |  11 +-
 .../apache/beam/sdk/options/PubsubOptions.java  |   2 +-
 .../beam/sdk/options/StreamingOptions.java      |   2 +-
 .../org/apache/beam/sdk/options/Validation.java |   2 +-
 .../apache/beam/sdk/options/package-info.java   |   8 +-
 .../java/org/apache/beam/sdk/package-info.java  |  12 +-
 .../runners/AggregatorPipelineExtractor.java    |  17 +-
 .../runners/AggregatorRetrievalException.java   |   4 +-
 .../beam/sdk/runners/AggregatorValues.java      |   8 +-
 .../apache/beam/sdk/runners/DirectPipeline.java |   8 +-
 .../sdk/runners/DirectPipelineRegistrar.java    |   9 +-
 .../beam/sdk/runners/DirectPipelineRunner.java  | 114 ++++-----
 .../apache/beam/sdk/runners/PipelineRunner.java |  23 +-
 .../sdk/runners/PipelineRunnerRegistrar.java    |   5 +-
 .../sdk/runners/RecordingPipelineVisitor.java   |  10 +-
 .../beam/sdk/runners/TransformHierarchy.java    |  11 +-
 .../beam/sdk/runners/TransformTreeNode.java     |  13 +-
 .../inprocess/AbstractModelEnforcement.java     |   6 +-
 .../inprocess/AvroIOShardedWriteFactory.java    |  16 +-
 .../inprocess/BoundedReadEvaluatorFactory.java  |  22 +-
 .../sdk/runners/inprocess/BundleFactory.java    |  12 +-
 .../CachedThreadPoolExecutorServiceFactory.java |   6 +-
 .../beam/sdk/runners/inprocess/Clock.java       |   2 +-
 .../runners/inprocess/CompletionCallback.java   |   4 +-
 .../ConsumerTrackingPipelineVisitor.java        |  20 +-
 .../inprocess/EmptyTransformEvaluator.java      |   8 +-
 .../EncodabilityEnforcementFactory.java         |  18 +-
 .../sdk/runners/inprocess/EvaluatorKey.java     |   4 +-
 .../inprocess/ExecutorServiceFactory.java       |   2 +-
 .../ExecutorServiceParallelExecutor.java        |  29 +--
 .../inprocess/FlattenEvaluatorFactory.java      |  20 +-
 .../runners/inprocess/ForwardingPTransform.java |  14 +-
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  55 ++---
 .../ImmutabilityCheckingBundleFactory.java      |  27 +--
 .../ImmutabilityEnforcementFactory.java         |  22 +-
 .../inprocess/InMemoryWatermarkManager.java     |  27 +--
 .../sdk/runners/inprocess/InProcessBundle.java  |  11 +-
 .../inprocess/InProcessBundleFactory.java       |  11 +-
 .../inprocess/InProcessBundleOutputManager.java |  12 +-
 .../sdk/runners/inprocess/InProcessCreate.java  |  35 +--
 .../inprocess/InProcessEvaluationContext.java   |  47 ++--
 .../inprocess/InProcessExecutionContext.java    |  18 +-
 .../runners/inprocess/InProcessExecutor.java    |   8 +-
 .../inprocess/InProcessPipelineOptions.java     |  18 +-
 .../inprocess/InProcessPipelineRunner.java      |  70 +++---
 .../runners/inprocess/InProcessRegistrar.java   |  11 +-
 .../inprocess/InProcessSideInputContainer.java  |  17 +-
 .../inprocess/InProcessTimerInternals.java      |  10 +-
 .../inprocess/InProcessTransformResult.java     |  16 +-
 .../inprocess/KeyedPValueTrackingVisitor.java   |  12 +-
 .../sdk/runners/inprocess/ModelEnforcement.java |  12 +-
 .../inprocess/ModelEnforcementFactory.java      |   6 +-
 .../sdk/runners/inprocess/NanosOffsetClock.java |   6 +-
 .../inprocess/PTransformOverrideFactory.java    |   8 +-
 .../inprocess/ParDoInProcessEvaluator.java      |  24 +-
 .../inprocess/ParDoMultiEvaluatorFactory.java   |  30 +--
 .../inprocess/ParDoSingleEvaluatorFactory.java  |  26 +--
 .../PassthroughTransformEvaluator.java          |   8 +-
 .../runners/inprocess/ShardControlledWrite.java |  12 +-
 .../beam/sdk/runners/inprocess/StepAndKey.java  |   5 +-
 .../runners/inprocess/StepTransformResult.java  |  15 +-
 .../inprocess/TextIOShardedWriteFactory.java    |  18 +-
 .../runners/inprocess/TransformEvaluator.java   |   6 +-
 .../inprocess/TransformEvaluatorFactory.java    |  10 +-
 .../inprocess/TransformEvaluatorRegistry.java   |  17 +-
 .../runners/inprocess/TransformExecutor.java    |   9 +-
 .../inprocess/TransformExecutorService.java     |   2 +-
 .../inprocess/TransformExecutorServices.java    |   2 +-
 .../UnboundedReadEvaluatorFactory.java          |  24 +-
 .../runners/inprocess/ViewEvaluatorFactory.java |  30 +--
 .../inprocess/WatermarkCallbackExecutor.java    |   9 +-
 .../inprocess/WindowEvaluatorFactory.java       |  26 +--
 .../apache/beam/sdk/runners/package-info.java   |  14 +-
 .../beam/sdk/runners/worker/IsmFormat.java      |  34 +--
 .../beam/sdk/runners/worker/package-info.java   |   2 +-
 .../beam/sdk/testing/CoderProperties.java       |  25 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  53 ++---
 .../beam/sdk/testing/RunnableOnService.java     |   2 +-
 .../beam/sdk/testing/SerializableMatcher.java   |   2 +-
 .../beam/sdk/testing/SerializableMatchers.java  |  19 +-
 .../beam/sdk/testing/SourceTestUtils.java       |  18 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |  25 +-
 .../beam/sdk/testing/WindowFnTestUtils.java     |  12 +-
 .../apache/beam/sdk/testing/package-info.java   |   2 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |   4 +-
 .../sdk/transforms/AggregatorRetriever.java     |   2 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |   7 +-
 .../sdk/transforms/ApproximateQuantiles.java    |  29 +--
 .../beam/sdk/transforms/ApproximateUnique.java  |  23 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  96 ++++----
 .../beam/sdk/transforms/CombineFnBase.java      |  19 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |  33 +--
 .../beam/sdk/transforms/CombineWithContext.java |  18 +-
 .../org/apache/beam/sdk/transforms/Count.java   |  20 +-
 .../org/apache/beam/sdk/transforms/Create.java  |  33 +--
 .../org/apache/beam/sdk/transforms/DoFn.java    |  43 ++--
 .../beam/sdk/transforms/DoFnReflector.java      |  37 +--
 .../apache/beam/sdk/transforms/DoFnTester.java  |  35 +--
 .../beam/sdk/transforms/DoFnWithContext.java    |  36 +--
 .../org/apache/beam/sdk/transforms/Filter.java  |   6 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   6 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |  22 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |  34 +--
 .../transforms/IntraBundleParallelization.java  |  25 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   8 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   8 +-
 .../apache/beam/sdk/transforms/MapElements.java |   6 +-
 .../org/apache/beam/sdk/transforms/Max.java     |  10 +-
 .../org/apache/beam/sdk/transforms/Mean.java    |  19 +-
 .../org/apache/beam/sdk/transforms/Min.java     |  10 +-
 .../apache/beam/sdk/transforms/PTransform.java  |  46 ++--
 .../org/apache/beam/sdk/transforms/ParDo.java   |  58 ++---
 .../apache/beam/sdk/transforms/Partition.java   |  16 +-
 .../beam/sdk/transforms/RemoveDuplicates.java   |  10 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |  25 +-
 .../sdk/transforms/SerializableComparator.java  |   2 +-
 .../sdk/transforms/SerializableFunction.java    |   2 +-
 .../beam/sdk/transforms/SimpleFunction.java     |   4 +-
 .../org/apache/beam/sdk/transforms/Sum.java     |   8 +-
 .../org/apache/beam/sdk/transforms/Top.java     |  31 +--
 .../org/apache/beam/sdk/transforms/Values.java  |   8 +-
 .../org/apache/beam/sdk/transforms/View.java    |  18 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |  18 +-
 .../beam/sdk/transforms/WithTimestamps.java     |  16 +-
 .../org/apache/beam/sdk/transforms/Write.java   |  10 +-
 .../sdk/transforms/display/DisplayData.java     |   9 +-
 .../sdk/transforms/display/HasDisplayData.java  |   4 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |  28 +--
 .../sdk/transforms/join/CoGbkResultSchema.java  |  12 +-
 .../beam/sdk/transforms/join/CoGroupByKey.java  |  30 +--
 .../transforms/join/KeyedPCollectionTuple.java  |  26 +--
 .../beam/sdk/transforms/join/RawUnionValue.java |   2 +-
 .../beam/sdk/transforms/join/UnionCoder.java    |  16 +-
 .../beam/sdk/transforms/join/package-info.java  |   4 +-
 .../beam/sdk/transforms/package-info.java       |  26 +--
 .../beam/sdk/transforms/windowing/AfterAll.java |   7 +-
 .../windowing/AfterDelayFromFirstElement.java   |  29 +--
 .../sdk/transforms/windowing/AfterEach.java     |   4 +-
 .../sdk/transforms/windowing/AfterFirst.java    |   7 +-
 .../sdk/transforms/windowing/AfterPane.java     |  20 +-
 .../windowing/AfterProcessingTime.java          |   6 +-
 .../AfterSynchronizedProcessingTime.java        |   7 +-
 .../transforms/windowing/AfterWatermark.java    |  13 +-
 .../sdk/transforms/windowing/BoundedWindow.java |   2 +-
 .../transforms/windowing/CalendarWindows.java   |   4 +-
 .../transforms/windowing/DefaultTrigger.java    |   4 +-
 .../sdk/transforms/windowing/FixedWindows.java  |   4 +-
 .../sdk/transforms/windowing/GlobalWindow.java  |   4 +-
 .../sdk/transforms/windowing/GlobalWindows.java |   4 +-
 .../transforms/windowing/IntervalWindow.java    |  12 +-
 .../transforms/windowing/InvalidWindows.java    |   4 +-
 .../MergeOverlappingIntervalWindows.java        |   2 +-
 .../windowing/NonMergingWindowFn.java           |   2 +-
 .../transforms/windowing/OrFinallyTrigger.java  |   5 +-
 .../sdk/transforms/windowing/OutputTimeFn.java  |   3 +-
 .../sdk/transforms/windowing/OutputTimeFns.java |   3 +-
 .../beam/sdk/transforms/windowing/PaneInfo.java |  25 +-
 .../windowing/PartitioningWindowFn.java         |   2 +-
 .../sdk/transforms/windowing/Repeatedly.java    |   4 +-
 .../beam/sdk/transforms/windowing/Sessions.java |   4 +-
 .../transforms/windowing/SlidingWindows.java    |   4 +-
 .../beam/sdk/transforms/windowing/Trigger.java  |  13 +-
 .../transforms/windowing/TriggerBuilder.java    |   2 +-
 .../beam/sdk/transforms/windowing/Window.java   |  40 ++--
 .../beam/sdk/transforms/windowing/WindowFn.java |   7 +-
 .../sdk/transforms/windowing/package-info.java  |  16 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   |   8 +-
 .../org/apache/beam/sdk/util/ApiSurface.java    |  10 +-
 .../beam/sdk/util/AppEngineEnvironment.java     |   2 +-
 .../apache/beam/sdk/util/AppliedCombineFn.java  |  19 +-
 .../org/apache/beam/sdk/util/AssignWindows.java |  12 +-
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |  10 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java |   2 +-
 .../util/AttemptBoundedExponentialBackOff.java  |   2 +-
 .../org/apache/beam/sdk/util/AvroUtils.java     |   2 +-
 .../beam/sdk/util/BaseExecutionContext.java     |  16 +-
 .../beam/sdk/util/BatchTimerInternals.java      |   5 +-
 .../apache/beam/sdk/util/BigQueryServices.java  |   5 +-
 .../beam/sdk/util/BigQueryServicesImpl.java     |   5 +-
 .../beam/sdk/util/BigQueryTableInserter.java    |  11 +-
 .../beam/sdk/util/BigQueryTableRowIterator.java |   2 +-
 .../org/apache/beam/sdk/util/BitSetCoder.java   |   8 +-
 .../BufferedElementCountingOutputStream.java    |   4 +-
 .../apache/beam/sdk/util/CloudKnownType.java    |   2 +-
 .../org/apache/beam/sdk/util/CloudObject.java   |   2 +-
 .../org/apache/beam/sdk/util/CoderUtils.java    |  25 +-
 .../beam/sdk/util/CombineContextFactory.java    |  14 +-
 .../org/apache/beam/sdk/util/CombineFnUtil.java |  24 +-
 .../apache/beam/sdk/util/CounterAggregator.java |  18 +-
 .../apache/beam/sdk/util/CredentialFactory.java |   2 +-
 .../org/apache/beam/sdk/util/Credentials.java   |   5 +-
 .../beam/sdk/util/DataflowReleaseInfo.java      |   4 +-
 .../sdk/util/DirectModeExecutionContext.java    |  13 +-
 .../beam/sdk/util/DirectSideInputReader.java    |  11 +-
 .../java/org/apache/beam/sdk/util/DoFnInfo.java |   8 +-
 .../org/apache/beam/sdk/util/DoFnRunner.java    |  10 +-
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  43 ++--
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  20 +-
 .../apache/beam/sdk/util/ExecutableTrigger.java |   9 +-
 .../apache/beam/sdk/util/ExecutionContext.java  |  20 +-
 .../sdk/util/ExposedByteArrayInputStream.java   |   2 +-
 .../sdk/util/ExposedByteArrayOutputStream.java  |   2 +-
 .../beam/sdk/util/FileIOChannelFactory.java     |   2 +-
 .../apache/beam/sdk/util/FinishedTriggers.java  |   2 +-
 .../beam/sdk/util/FinishedTriggersBitSet.java   |   2 +-
 .../beam/sdk/util/FinishedTriggersSet.java      |   2 +-
 .../beam/sdk/util/GcpCredentialFactory.java     |   7 +-
 .../beam/sdk/util/GcsIOChannelFactory.java      |   6 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  11 +-
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |  18 +-
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |  14 +-
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  11 +-
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  26 +--
 .../apache/beam/sdk/util/IOChannelFactory.java  |   2 +-
 .../apache/beam/sdk/util/IOChannelUtils.java    |   6 +-
 .../beam/sdk/util/IllegalMutationException.java |   2 +-
 .../apache/beam/sdk/util/InstanceBuilder.java   |   5 +-
 .../util/IntervalBoundedExponentialBackOff.java |   2 +-
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |   4 +-
 .../beam/sdk/util/KeyedWorkItemCoder.java       |  19 +-
 .../apache/beam/sdk/util/KeyedWorkItems.java    |   5 +-
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  11 +-
 .../beam/sdk/util/MapAggregatorValues.java      |   7 +-
 .../beam/sdk/util/MergingActiveWindowSet.java   |  23 +-
 .../org/apache/beam/sdk/util/MimeTypes.java     |   2 +-
 .../apache/beam/sdk/util/MutationDetector.java  |   2 +-
 .../apache/beam/sdk/util/MutationDetectors.java |   6 +-
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  26 +--
 .../sdk/util/NonMergingActiveWindowSet.java     |   7 +-
 .../beam/sdk/util/NoopCredentialFactory.java    |   5 +-
 .../apache/beam/sdk/util/NoopPathValidator.java |   4 +-
 .../beam/sdk/util/NullSideInputReader.java      |   7 +-
 .../apache/beam/sdk/util/OutputReference.java   |   2 +-
 .../beam/sdk/util/PCollectionViewWindow.java    |   6 +-
 .../apache/beam/sdk/util/PCollectionViews.java  |  25 +-
 .../java/org/apache/beam/sdk/util/PTuple.java   |   4 +-
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  23 +-
 .../org/apache/beam/sdk/util/PathValidator.java |   2 +-
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  10 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |  21 +-
 .../org/apache/beam/sdk/util/PropertyNames.java |   2 +-
 .../apache/beam/sdk/util/RandomAccessData.java  |  11 +-
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  12 +-
 .../beam/sdk/util/ReduceFnContextFactory.java   |  35 +--
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  44 ++--
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |   6 +-
 .../org/apache/beam/sdk/util/Reshuffle.java     |  26 +--
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |   6 +-
 .../sdk/util/RetryHttpRequestInitializer.java   |   2 +-
 .../apache/beam/sdk/util/SerializableUtils.java |  11 +-
 .../org/apache/beam/sdk/util/Serializer.java    |   2 +-
 .../sdk/util/ShardingWritableByteChannel.java   |   2 +-
 .../apache/beam/sdk/util/SideInputReader.java   |   6 +-
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |  14 +-
 .../org/apache/beam/sdk/util/StreamUtils.java   |   2 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   5 +-
 .../java/org/apache/beam/sdk/util/Structs.java  |   2 +-
 .../beam/sdk/util/SystemDoFnInternal.java       |   4 +-
 .../apache/beam/sdk/util/SystemReduceFn.java    |  32 +--
 .../apache/beam/sdk/util/TestCredential.java    |   2 +-
 .../org/apache/beam/sdk/util/TimeDomain.java    |   2 +-
 .../java/org/apache/beam/sdk/util/TimeUtil.java |   2 +-
 .../apache/beam/sdk/util/TimerInternals.java    |  19 +-
 .../java/org/apache/beam/sdk/util/Timers.java   |   2 +-
 .../org/apache/beam/sdk/util/Transport.java     |   9 +-
 .../beam/sdk/util/TriggerContextFactory.java    |  29 +--
 .../org/apache/beam/sdk/util/TriggerRunner.java |  21 +-
 .../beam/sdk/util/UnownedInputStream.java       |   2 +-
 .../beam/sdk/util/UnownedOutputStream.java      |   2 +-
 .../sdk/util/UploadIdResponseInterceptor.java   |   2 +-
 .../apache/beam/sdk/util/UserCodeException.java |   2 +-
 .../apache/beam/sdk/util/ValueWithRecordId.java |  19 +-
 .../java/org/apache/beam/sdk/util/Values.java   |   2 +-
 .../java/org/apache/beam/sdk/util/VarInt.java   |   2 +-
 .../org/apache/beam/sdk/util/WatermarkHold.java |  25 +-
 .../java/org/apache/beam/sdk/util/Weighted.java |   2 +-
 .../org/apache/beam/sdk/util/WeightedValue.java |   2 +-
 .../org/apache/beam/sdk/util/WindowTracing.java |   2 +-
 .../org/apache/beam/sdk/util/WindowedValue.java |  26 ++-
 .../beam/sdk/util/WindowingInternals.java       |  14 +-
 .../apache/beam/sdk/util/WindowingStrategy.java |  19 +-
 .../java/org/apache/beam/sdk/util/ZipFiles.java |   2 +-
 .../apache/beam/sdk/util/common/Counter.java    |  12 +-
 .../beam/sdk/util/common/CounterProvider.java   |   2 +-
 .../apache/beam/sdk/util/common/CounterSet.java |   2 +-
 .../util/common/ElementByteSizeObservable.java  |   2 +-
 .../ElementByteSizeObservableIterable.java      |   2 +-
 .../ElementByteSizeObservableIterator.java      |   2 +-
 .../util/common/ElementByteSizeObserver.java    |   2 +-
 .../beam/sdk/util/common/PeekingReiterator.java |   2 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |   2 +-
 .../apache/beam/sdk/util/common/Reiterable.java |   2 +-
 .../apache/beam/sdk/util/common/Reiterator.java |   2 +-
 .../beam/sdk/util/common/package-info.java      |   2 +-
 .../sdk/util/common/worker/StateSampler.java    |   7 +-
 .../sdk/util/common/worker/package-info.java    |   2 +-
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java |   2 +-
 .../beam/sdk/util/gcsfs/package-info.java       |   2 +-
 .../org/apache/beam/sdk/util/package-info.java  |   2 +-
 .../util/state/AccumulatorCombiningState.java   |   4 +-
 .../apache/beam/sdk/util/state/BagState.java    |   2 +-
 .../beam/sdk/util/state/CombiningState.java     |   4 +-
 .../CopyOnAccessInMemoryStateInternals.java     |  21 +-
 .../sdk/util/state/InMemoryStateInternals.java  |  18 +-
 .../sdk/util/state/MergingStateAccessor.java    |   4 +-
 .../beam/sdk/util/state/ReadableState.java      |   2 +-
 .../org/apache/beam/sdk/util/state/State.java   |   2 +-
 .../beam/sdk/util/state/StateAccessor.java      |   2 +-
 .../beam/sdk/util/state/StateContext.java       |  10 +-
 .../beam/sdk/util/state/StateContexts.java      |  10 +-
 .../beam/sdk/util/state/StateInternals.java     |   4 +-
 .../beam/sdk/util/state/StateMerging.java       |   5 +-
 .../beam/sdk/util/state/StateNamespace.java     |   2 +-
 .../sdk/util/state/StateNamespaceForTest.java   |   2 +-
 .../beam/sdk/util/state/StateNamespaces.java    |  11 +-
 .../apache/beam/sdk/util/state/StateTable.java  |   5 +-
 .../apache/beam/sdk/util/state/StateTag.java    |  16 +-
 .../apache/beam/sdk/util/state/StateTags.java   |  19 +-
 .../apache/beam/sdk/util/state/ValueState.java  |   2 +-
 .../beam/sdk/util/state/WatermarkHoldState.java |   6 +-
 .../java/org/apache/beam/sdk/values/KV.java     |  12 +-
 .../java/org/apache/beam/sdk/values/PBegin.java |  10 +-
 .../org/apache/beam/sdk/values/PCollection.java |  32 +--
 .../apache/beam/sdk/values/PCollectionList.java |  13 +-
 .../beam/sdk/values/PCollectionTuple.java       |  15 +-
 .../apache/beam/sdk/values/PCollectionView.java |  16 +-
 .../java/org/apache/beam/sdk/values/PDone.java  |   6 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   8 +-
 .../org/apache/beam/sdk/values/POutput.java     |   8 +-
 .../beam/sdk/values/POutputValueBase.java       |   8 +-
 .../java/org/apache/beam/sdk/values/PValue.java |   6 +-
 .../org/apache/beam/sdk/values/PValueBase.java  |   9 +-
 .../beam/sdk/values/TimestampedValue.java       |  10 +-
 .../org/apache/beam/sdk/values/TupleTag.java    |  12 +-
 .../apache/beam/sdk/values/TupleTagList.java    |   5 +-
 .../apache/beam/sdk/values/TypeDescriptor.java  |   2 +-
 .../org/apache/beam/sdk/values/TypedPValue.java |  20 +-
 .../apache/beam/sdk/values/package-info.java    |  36 +--
 .../main/proto/proto2_coder_test_messages.proto |   2 +-
 .../google/cloud/dataflow/sdk/sdk.properties    |   5 -
 .../org/apache/beam/sdk/sdk.properties          |   5 +
 .../org/apache/beam/sdk/DataflowMatchers.java   |   2 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |  49 ++--
 .../java/org/apache/beam/sdk/TestUtils.java     |   6 +-
 .../org/apache/beam/sdk/WindowMatchers.java     |  10 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  26 +--
 .../sdk/coders/BigEndianIntegerCoderTest.java   |   8 +-
 .../beam/sdk/coders/BigEndianLongCoderTest.java |   8 +-
 .../beam/sdk/coders/ByteArrayCoderTest.java     |  10 +-
 .../apache/beam/sdk/coders/ByteCoderTest.java   |   8 +-
 .../beam/sdk/coders/ByteStringCoderTest.java    |  11 +-
 .../beam/sdk/coders/CoderFactoriesTest.java     |   2 +-
 .../beam/sdk/coders/CoderProvidersTest.java     |   4 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |  31 +--
 .../org/apache/beam/sdk/coders/CoderTest.java   |   6 +-
 .../beam/sdk/coders/CollectionCoderTest.java    |   8 +-
 .../apache/beam/sdk/coders/CustomCoderTest.java |  10 +-
 .../beam/sdk/coders/DefaultCoderTest.java       |   5 +-
 .../beam/sdk/coders/DelegateCoderTest.java      |   5 +-
 .../apache/beam/sdk/coders/DoubleCoderTest.java |   8 +-
 .../beam/sdk/coders/DurationCoderTest.java      |   9 +-
 .../apache/beam/sdk/coders/EntityCoderTest.java |   9 +-
 .../beam/sdk/coders/InstantCoderTest.java       |   9 +-
 .../beam/sdk/coders/IterableCoderTest.java      |   8 +-
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |   7 +-
 .../org/apache/beam/sdk/coders/KvCoderTest.java |  11 +-
 .../apache/beam/sdk/coders/ListCoderTest.java   |   8 +-
 .../apache/beam/sdk/coders/MapCoderTest.java    |   9 +-
 .../beam/sdk/coders/NullableCoderTest.java      |   7 +-
 .../beam/sdk/coders/PrintBase64Encodings.java   |  11 +-
 .../apache/beam/sdk/coders/Proto2CoderTest.java |  10 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |  24 +-
 .../apache/beam/sdk/coders/SetCoderTest.java    |   8 +-
 .../beam/sdk/coders/StandardCoderTest.java      |   8 +-
 .../sdk/coders/StringDelegateCoderTest.java     |   6 +-
 .../beam/sdk/coders/StringUtf8CoderTest.java    |   8 +-
 .../sdk/coders/StructuralByteArrayTest.java     |   2 +-
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |   7 +-
 .../sdk/coders/TextualIntegerCoderTest.java     |   8 +-
 .../apache/beam/sdk/coders/VarIntCoderTest.java |   8 +-
 .../beam/sdk/coders/VarLongCoderTest.java       |   8 +-
 .../sdk/coders/protobuf/ProtoCoderTest.java     |  19 +-
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |  12 +-
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  14 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  21 +-
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  17 +-
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  31 +--
 .../io/BoundedReadFromUnboundedSourceTest.java  |  20 +-
 .../beam/sdk/io/CompressedSourceTest.java       |  25 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  28 +--
 .../apache/beam/sdk/io/CountingSourceTest.java  |  38 +--
 .../org/apache/beam/sdk/io/DatastoreIOTest.java |  18 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  12 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  38 +--
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  15 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |   2 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |   8 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  46 ++--
 .../java/org/apache/beam/sdk/io/WriteTest.java  |  31 +--
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  11 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  24 +-
 .../beam/sdk/io/bigtable/BigtableIOTest.java    |  37 +--
 .../range/ByteKeyRangeEstimateFractionTest.java |   2 +-
 .../range/ByteKeyRangeInterpolateKeyTest.java   |   2 +-
 .../beam/sdk/io/range/ByteKeyRangeTest.java     |   2 +-
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |   2 +-
 .../apache/beam/sdk/io/range/ByteKeyTest.java   |   2 +-
 .../sdk/io/range/OffsetRangeTrackerTest.java    |   2 +-
 .../test/java/org/apache/beam/sdk/io/user.avsc  |   2 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |   7 +-
 .../sdk/options/GoogleApiDebugOptionsTest.java  |   9 +-
 .../sdk/options/PipelineOptionsFactoryTest.java | 104 ++++-----
 .../beam/sdk/options/PipelineOptionsTest.java   |   5 +-
 .../options/PipelineOptionsValidatorTest.java   |  10 +-
 .../sdk/options/ProxyInvocationHandlerTest.java |   6 +-
 .../AggregatorPipelineExtractorTest.java        |  23 +-
 .../runners/DirectPipelineRegistrarTest.java    |   7 +-
 .../sdk/runners/DirectPipelineRunnerTest.java   |  35 +--
 .../beam/sdk/runners/DirectPipelineTest.java    |   2 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |  12 +-
 .../beam/sdk/runners/TransformTreeTest.java     |  34 +--
 .../runners/dataflow/TestCountingSource.java    |  18 +-
 .../AvroIOShardedWriteFactoryTest.java          |  16 +-
 .../BoundedReadEvaluatorFactoryTest.java        |  33 +--
 .../ConsumerTrackingPipelineVisitorTest.java    |  32 +--
 .../EncodabilityEnforcementFactoryTest.java     |  40 ++--
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  22 +-
 .../inprocess/ForwardingPTransformTest.java     |   8 +-
 .../GroupByKeyEvaluatorFactoryTest.java         |  27 +--
 .../ImmutabilityCheckingBundleFactoryTest.java  |  28 +--
 .../ImmutabilityEnforcementFactoryTest.java     |  22 +-
 .../inprocess/InMemoryWatermarkManagerTest.java |  51 +++--
 .../inprocess/InProcessBundleFactoryTest.java   |  19 +-
 .../runners/inprocess/InProcessBundleTest.java  |  13 +-
 .../runners/inprocess/InProcessCreateTest.java  |  45 ++--
 .../InProcessEvaluationContextTest.java         |  75 +++---
 .../InProcessPipelineRegistrarTest.java         |   9 +-
 .../inprocess/InProcessPipelineRunnerTest.java  |  24 +-
 .../InProcessSideInputContainerTest.java        |  39 ++--
 .../inprocess/InProcessTimerInternalsTest.java  |  14 +-
 .../KeyedPValueTrackingVisitorTest.java         |  33 +--
 .../beam/sdk/runners/inprocess/MockClock.java   |   2 +-
 .../ParDoMultiEvaluatorFactoryTest.java         |  68 +++---
 .../ParDoSingleEvaluatorFactoryTest.java        |  58 ++---
 .../TextIOShardedWriteFactoryTest.java          |  18 +-
 .../TransformExecutorServicesTest.java          |   2 +-
 .../inprocess/TransformExecutorTest.java        |  25 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |  43 ++--
 .../inprocess/ViewEvaluatorFactoryTest.java     |  33 +--
 .../WatermarkCallbackExecutorTest.java          |  24 +-
 .../inprocess/WindowEvaluatorFactoryTest.java   |  35 +--
 .../beam/sdk/testing/CoderPropertiesTest.java   |  11 +-
 .../sdk/testing/DataflowJUnitTestRunner.java    |  11 +-
 .../apache/beam/sdk/testing/ExpectedLogs.java   |   2 +-
 .../beam/sdk/testing/ExpectedLogsTest.java      |   4 +-
 .../sdk/testing/FastNanoClockAndSleeper.java    |   2 +-
 .../testing/FastNanoClockAndSleeperTest.java    |   2 +-
 .../apache/beam/sdk/testing/PAssertTest.java    |  21 +-
 .../sdk/testing/PCollectionViewTesting.java     |  29 +--
 .../beam/sdk/testing/ResetDateTimeProvider.java |   2 +-
 .../sdk/testing/ResetDateTimeProviderTest.java  |   2 +-
 .../sdk/testing/RestoreSystemProperties.java    |   2 +-
 .../testing/RestoreSystemPropertiesTest.java    |   2 +-
 .../sdk/testing/SerializableMatchersTest.java   |  22 +-
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |   2 +-
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |   5 +-
 .../beam/sdk/testing/TestPipelineTest.java      |  10 +-
 .../transforms/ApproximateQuantilesTest.java    |  25 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |  19 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |  43 ++--
 .../apache/beam/sdk/transforms/CombineTest.java |  75 +++---
 .../apache/beam/sdk/transforms/CountTest.java   |  19 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |  31 +--
 .../beam/sdk/transforms/DoFnContextTest.java    |   2 +-
 .../DoFnDelegatingAggregatorTest.java           |   6 +-
 .../beam/sdk/transforms/DoFnReflectorTest.java  |  14 +-
 .../apache/beam/sdk/transforms/DoFnTest.java    |  12 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |   4 +-
 .../sdk/transforms/DoFnWithContextTest.java     |  10 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |  10 +-
 .../sdk/transforms/FlatMapElementsTest.java     |  15 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |  51 +++--
 .../beam/sdk/transforms/GroupByKeyTest.java     |  49 ++--
 .../IntraBundleParallelizationTest.java         |   9 +-
 .../apache/beam/sdk/transforms/KeysTest.java    |  20 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |  20 +-
 .../beam/sdk/transforms/MapElementsTest.java    |  14 +-
 .../org/apache/beam/sdk/transforms/MaxTest.java |   5 +-
 .../apache/beam/sdk/transforms/MeanTest.java    |  14 +-
 .../org/apache/beam/sdk/transforms/MinTest.java |   5 +-
 .../apache/beam/sdk/transforms/NoOpDoFn.java    |  18 +-
 .../beam/sdk/transforms/PTransformTest.java     |   6 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  62 ++---
 .../beam/sdk/transforms/PartitionTest.java      |  16 +-
 .../sdk/transforms/RemoveDuplicatesTest.java    |  16 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |  22 +-
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |   2 +-
 .../org/apache/beam/sdk/transforms/SumTest.java |   5 +-
 .../org/apache/beam/sdk/transforms/TopTest.java |  24 +-
 .../apache/beam/sdk/transforms/ValuesTest.java  |  20 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |  59 ++---
 .../beam/sdk/transforms/WithKeysTest.java       |  16 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |  14 +-
 .../transforms/display/DisplayDataMatchers.java |   5 +-
 .../display/DisplayDataMatchersTest.java        |  23 +-
 .../sdk/transforms/display/DisplayDataTest.java |  27 ++-
 .../transforms/join/CoGbkResultCoderTest.java   |  21 +-
 .../sdk/transforms/join/CoGbkResultTest.java    |  10 +-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  41 ++--
 .../sdk/transforms/join/UnionCoderTest.java     |  12 +-
 .../sdk/transforms/windowing/AfterAllTest.java  |   8 +-
 .../sdk/transforms/windowing/AfterEachTest.java |   8 +-
 .../transforms/windowing/AfterFirstTest.java    |   8 +-
 .../sdk/transforms/windowing/AfterPaneTest.java |   6 +-
 .../windowing/AfterProcessingTimeTest.java      |   8 +-
 .../AfterSynchronizedProcessingTimeTest.java    |   6 +-
 .../windowing/AfterWatermarkTest.java           |   8 +-
 .../windowing/CalendarWindowsTest.java          |   7 +-
 .../windowing/DefaultTriggerTest.java           |   6 +-
 .../transforms/windowing/FixedWindowsTest.java  |   9 +-
 .../windowing/IntervalWindowTest.java           |  11 +-
 .../windowing/OrFinallyTriggerTest.java         |   8 +-
 .../sdk/transforms/windowing/PaneInfoTest.java  |   8 +-
 .../transforms/windowing/RepeatedlyTest.java    |   6 +-
 .../sdk/transforms/windowing/SessionsTest.java  |   9 +-
 .../windowing/SlidingWindowsTest.java           |   9 +-
 .../sdk/transforms/windowing/TriggerTest.java   |   2 +-
 .../sdk/transforms/windowing/WindowTest.java    |  30 +--
 .../sdk/transforms/windowing/WindowingTest.java |  38 +--
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  16 +-
 ...mptAndTimeBoundedExponentialBackOffTest.java |   5 +-
 .../AttemptBoundedExponentialBackOffTest.java   |   3 +-
 .../org/apache/beam/sdk/util/AvroUtilsTest.java |   9 +-
 .../beam/sdk/util/BatchTimerInternalsTest.java  |   8 +-
 .../beam/sdk/util/BigQueryServicesImplTest.java |   7 +-
 .../sdk/util/BigQueryTableInserterTest.java     |   6 +-
 .../sdk/util/BigQueryTableRowIteratorTest.java  |   2 +-
 .../apache/beam/sdk/util/BigQueryUtilTest.java  |  11 +-
 ...BufferedElementCountingOutputStreamTest.java |   7 +-
 .../apache/beam/sdk/util/CoderUtilsTest.java    |  25 +-
 .../apache/beam/sdk/util/CombineFnUtilTest.java |  13 +-
 .../beam/sdk/util/CounterAggregatorTest.java    |  34 +--
 .../beam/sdk/util/ExecutableTriggerTest.java    |   6 +-
 .../util/ExposedByteArrayInputStreamTest.java   |   2 +-
 .../util/ExposedByteArrayOutputStreamTest.java  |   2 +-
 .../beam/sdk/util/FileIOChannelFactoryTest.java |   2 +-
 .../sdk/util/FinishedTriggersBitSetTest.java    |   2 +-
 .../sdk/util/FinishedTriggersProperties.java    |  12 +-
 .../beam/sdk/util/FinishedTriggersSetTest.java  |   2 +-
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |   6 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  11 +-
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  33 +--
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  10 +-
 .../beam/sdk/util/IOChannelUtilsTest.java       |   2 +-
 .../beam/sdk/util/InstanceBuilderTest.java      |   4 +-
 .../IntervalBoundedExponentialBackOffTest.java  |   2 +-
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |  15 +-
 .../util/LateDataDroppingDoFnRunnerTest.java    |  15 +-
 .../sdk/util/MergingActiveWindowSetTest.java    |  13 +-
 .../beam/sdk/util/MutationDetectorsTest.java    |  11 +-
 .../org/apache/beam/sdk/util/PTupleTest.java    |   4 +-
 .../beam/sdk/util/RandomAccessDataTest.java     |  11 +-
 .../beam/sdk/util/ReduceFnRunnerTest.java       |  58 ++---
 .../apache/beam/sdk/util/ReduceFnTester.java    |  69 +++---
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  31 +--
 .../beam/sdk/util/ReshuffleTriggerTest.java     |   8 +-
 .../util/RetryHttpRequestInitializerTest.java   |   2 +-
 .../beam/sdk/util/SerializableUtilsTest.java    |   9 +-
 .../apache/beam/sdk/util/SerializerTest.java    |  10 +-
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   8 +-
 .../apache/beam/sdk/util/StreamUtilsTest.java   |   2 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |  10 +-
 .../org/apache/beam/sdk/util/StructsTest.java   |  36 +--
 .../org/apache/beam/sdk/util/TimeUtilTest.java  |  11 +-
 .../beam/sdk/util/TimerInternalsTest.java       |  16 +-
 .../org/apache/beam/sdk/util/TriggerTester.java |  42 ++--
 .../beam/sdk/util/UnownedInputStreamTest.java   |   2 +-
 .../beam/sdk/util/UnownedOutputStreamTest.java  |   2 +-
 .../util/UploadIdResponseInterceptorTest.java   |   6 +-
 .../beam/sdk/util/UserCodeExceptionTest.java    |   2 +-
 .../org/apache/beam/sdk/util/VarIntTest.java    |   2 +-
 .../apache/beam/sdk/util/WindowedValueTest.java |  12 +-
 .../org/apache/beam/sdk/util/ZipFilesTest.java  |   2 +-
 .../beam/sdk/util/common/CounterSetTest.java    |   7 +-
 .../beam/sdk/util/common/CounterTest.java       |  21 +-
 .../beam/sdk/util/common/CounterTestUtils.java  |   8 +-
 .../sdk/util/common/ReflectHelpersTest.java     |   4 +-
 .../apache/beam/sdk/util/gcsfs/GcsPathTest.java |   2 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |  26 +--
 .../util/state/InMemoryStateInternalsTest.java  |  14 +-
 .../sdk/util/state/StateNamespacesTest.java     |  10 +-
 .../beam/sdk/util/state/StateTagTest.java       |  24 +-
 .../java/org/apache/beam/sdk/values/KVTest.java |   2 +-
 .../beam/sdk/values/PCollectionListTest.java    |   2 +-
 .../beam/sdk/values/PCollectionTupleTest.java   |  20 +-
 .../org/apache/beam/sdk/values/PDoneTest.java   |  16 +-
 .../apache/beam/sdk/values/TupleTagTest.java    |  14 +-
 .../beam/sdk/values/TypeDescriptorTest.java     |   2 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |  14 +-
 .../PipelineOptionsFactoryJava8Test.java        |  92 --------
 .../PipelineOptionsFactoryJava8Test.java        |  92 ++++++++
 .../beam/sdk/transforms/CombineJava8Test.java   |  12 +-
 .../beam/sdk/transforms/FilterJava8Test.java    |  16 +-
 .../transforms/FlatMapElementsJava8Test.java    |  13 +-
 .../sdk/transforms/MapElementsJava8Test.java    |  12 +-
 .../beam/sdk/transforms/PartitionJava8Test.java |  12 +-
 .../transforms/RemoveDuplicatesJava8Test.java   |  11 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |  16 +-
 .../sdk/transforms/WithTimestampsJava8Test.java |  12 +-
 .../src/main/java/DebuggingWordCount.java       |  32 +--
 .../src/main/java/MinimalWordCount.java         |  18 +-
 .../src/main/java/WindowedWordCount.java        |  28 +--
 .../src/main/java/WordCount.java                |  34 +--
 .../java/common/DataflowExampleOptions.java     |   6 +-
 .../main/java/common/DataflowExampleUtils.java  |  22 +-
 .../common/ExampleBigQueryTableOptions.java     |  10 +-
 .../java/common/ExamplePubsubTopicOptions.java  |  10 +-
 .../main/java/common/PubsubFileInjector.java    |  20 +-
 .../src/test/java/WordCountTest.java            |  18 +-
 .../src/main/java/StarterPipeline.java          |  10 +-
 .../src/main/java/it/pkg/StarterPipeline.java   |  10 +-
 travis/test_wordcount.sh                        |   4 +-
 976 files changed, 7700 insertions(+), 7161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/hadoop/pom.xml b/contrib/hadoop/pom.xml
index cf88019..24e454b 100644
--- a/contrib/hadoop/pom.xml
+++ b/contrib/hadoop/pom.xml
@@ -37,7 +37,7 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <google-cloud-dataflow-version>[1.2.0,2.0.0)</google-cloud-dataflow-version>
+    <beam-version>[0.1.0, 1.0.0)</beam-version>
   </properties>
 
   <build>
@@ -137,9 +137,9 @@
 
   <dependencies>
     <dependency>
-      <groupId>com.google.cloud.dataflow</groupId>
-      <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
-      <version>${google-cloud-dataflow-version}</version>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>java-sdk-all</artifactId>
+      <version>${beam-version}</version>
     </dependency>
 
     <!-- @tomwhite: Hadoop doesn't have great RPC client compatibility between one version and

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
index c8da9d2..65862f7 100644
--- a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
+++ b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/HadoopFileSource.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.hadoop;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.KV;
+package org.apache.beam.contrib.hadoop;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+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;
@@ -56,14 +56,14 @@ import javax.annotation.Nullable;
  * A {@code BoundedSource} for reading files resident in a Hadoop filesystem using a
  * Hadoop file-based input format.
  *
- * <p>To read a {@link com.google.cloud.dataflow.sdk.values.PCollection} of
- * {@link com.google.cloud.dataflow.sdk.values.KV} key-value pairs from one or more
+ * <p>To read a {@link org.apache.beam.sdk.values.PCollection} of
+ * {@link org.apache.beam.sdk.values.KV} key-value pairs from one or more
  * Hadoop files, use {@link HadoopFileSource#from} to specify the path(s) of the files to
  * read, the Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}, the
  * key class and the value class.
  *
  * <p>A {@code HadoopFileSource} can be read from using the
- * {@link com.google.cloud.dataflow.sdk.io.Read} transform. For example:
+ * {@link org.apache.beam.sdk.io.Read} transform. For example:
  *
  * <pre>
  * {@code
@@ -85,7 +85,7 @@ import javax.annotation.Nullable;
  *
  * Implementation note: Since Hadoop's {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
  * determines the input splits, this class extends {@link BoundedSource} rather than
- * {@link com.google.cloud.dataflow.sdk.io.OffsetBasedSource}, since the latter
+ * {@link org.apache.beam.sdk.io.OffsetBasedSource}, since the latter
  * dictates input splits.
 
  * @param <K> The type of keys to be read from the source.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
index 572c24b..180875d 100644
--- a/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
+++ b/contrib/hadoop/src/main/java/org/apache/contrib/hadoop/WritableCoder.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.hadoop;
+package org.apache.beam.contrib.hadoop;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
+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 com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.hadoop.io.Writable;
@@ -32,7 +32,7 @@ import java.io.OutputStream;
 import java.util.List;
 
 /**
- * A {@code WritableCoder} is a {@link com.google.cloud.dataflow.sdk.coders.Coder} for a
+ * A {@code WritableCoder} is a {@link org.apache.beam.sdk.coders.Coder} for a
  * Java class that implements {@link org.apache.hadoop.io.Writable}.
  *
  * <p> To use, specify the coder type on a PCollection:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
index 4aeb0f8..72bd72a 100644
--- a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
+++ b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/HadoopFileSourceTest.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.hadoop;
+package org.apache.beam.contrib.hadoop;
 
-import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
+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;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.SourceTestUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+import org.apache.beam.sdk.values.KV;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
index fac3fe2..368b682 100644
--- a/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
+++ b/contrib/hadoop/src/test/java/org/apache/contrib/hadoop/WritableCoderTest.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.hadoop;
+package org.apache.beam.contrib.hadoop;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.CoderProperties;
 
 import org.apache.hadoop.io.IntWritable;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/join-library/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/join-library/pom.xml b/contrib/join-library/pom.xml
index 10f663d..090f445 100644
--- a/contrib/join-library/pom.xml
+++ b/contrib/join-library/pom.xml
@@ -50,7 +50,7 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <google-cloud-dataflow-version>[1.0.0, 2.0.0)</google-cloud-dataflow-version>
+    <beam-version>[0.1.0, 1.0.0)</beam-version>
   </properties>
 
   <build>
@@ -155,9 +155,9 @@
 
   <dependencies>
     <dependency>
-      <groupId>com.google.cloud.dataflow</groupId>
-      <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
-      <version>${google-cloud-dataflow-version}</version>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>java-sdk-all</artifactId>
+      <version>${beam-version}</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java b/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
index b7f0df7..6421e97 100644
--- a/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
+++ b/contrib/join-library/src/main/java/org/apache/contrib/joinlibrary/Join.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.contrib.joinlibrary;
+
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+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;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
index c479a65..99e9c4b 100644
--- a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.contrib.joinlibrary;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
index 35c8655..ca09136 100644
--- a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.contrib.joinlibrary;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
index 9cc6785..86028ac 100644
--- a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.contrib.joinlibrary;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index bcac57d..f0ed060 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -104,8 +104,8 @@
           <windowtitle>Apache Beam Examples</windowtitle>
           <doctitle>Apache Beam Examples</doctitle>
 
-          <subpackages>com.google.cloud.dataflow.examples</subpackages>
-          <additionalparam>-exclude com.google.cloud.dataflow.sdk.runners.worker:com.google.cloud.dataflow.sdk.runners.dataflow:com.google.cloud.dataflow.sdk.util ${dataflow.javadoc_opts}</additionalparam>
+          <subpackages>org.apache.beam.examples</subpackages>
+          <additionalparam>-exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util ${dataflow.javadoc_opts}</additionalparam>
           <use>false</use>
           <quiet>true</quiet>
           <bottom><![CDATA[<br>]]></bottom>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7134bca..2e48465 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
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+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;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +70,7 @@ import java.util.regex.Pattern;
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
  * </pre>
  *
@@ -79,7 +79,7 @@ import java.util.regex.Pattern;
  * <pre>
  * mvn compile exec:java ... \
  *   -Dexec.args="... \
- *     --workerLogLevelOverrides={\\\"com.google.cloud.dataflow.examples\\\":\\\"DEBUG\\\"}"
+ *     --workerLogLevelOverrides={\\\"org.apache.beam.examples\\\":\\\"DEBUG\\\"}"
  * </pre>
  *
  * <p>Concept #2: Dataflow workers which execute user code are configured to log to Cloud
@@ -90,10 +90,10 @@ import java.util.regex.Pattern;
  * </code></pre>
  * For example, by specifying:
  * <pre><code>
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * </code></pre>
  * when executing this pipeline using the Dataflow service, Cloud Logging would contain only
- * "DEBUG" or higher level logs for the {@code com.google.cloud.dataflow.examples} package in
+ * "DEBUG" or higher level logs for the {@code org.apache.beam.examples} package in
  * addition to the default "INFO" or higher level logs. In addition, the default Dataflow worker
  * logging configuration can be overridden by specifying
  * {@code --defaultWorkerLogLevel=<one of TRACE, DEBUG, INFO, WARN, ERROR>}. For example,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f3be5a3..6c33ca6 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
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
index 8f49dd2..ecb837e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
@@ -15,30 +15,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
+
+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.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;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ad08d13..9fb9856 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
@@ -15,27 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.examples;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+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.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
index 557e903..5c62146 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
 
 /**
  * Options that can be used to configure the Dataflow examples.



[71/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
index 5b98170..93c2358 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -15,7 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.runners.DataflowPipelineJob;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.IntraBundleParallelization;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
@@ -34,21 +50,6 @@ import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.model.Subscription;
 import com.google.api.services.pubsub.model.Topic;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Strings;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
index a026f3e..647d508 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
+
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 
 /**
  * Options that can be used to configure BigQuery tables in Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
index bd8cace..b3da88d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * Options that can be used to configure Pub/Sub topic/subscription in Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
index 14df8d9..17f5b28 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * Options that can be used to configure Pub/Sub topic in Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
index eb3d1ac..b71b6f3 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.common;
+package org.apache.beam.examples.common;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.IntraBundleParallelization;
+import org.apache.beam.sdk.util.Transport;
 
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.model.PublishRequest;
 import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.util.Transport;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 343b17b..e1b658b 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
@@ -15,7 +15,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+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.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.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Partition;
+import org.apache.beam.sdk.transforms.Partition.PartitionFn;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+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;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableReference;
@@ -25,39 +59,6 @@ import com.google.api.services.datastore.DatastoreV1.Entity;
 import com.google.api.services.datastore.DatastoreV1.Key;
 import com.google.api.services.datastore.DatastoreV1.Value;
 import com.google.api.services.datastore.client.DatastoreHelper;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Top;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c61873c..7107e34 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
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+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.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
 
 import java.io.IOException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 79ce823..4b0db93 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
@@ -15,42 +15,42 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.examples.complete;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+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.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f1d8d1a..fa587ea 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
@@ -15,31 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+package org.apache.beam.examples.complete;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableComparator;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.CalendarWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableComparator;
-import com.google.cloud.dataflow.sdk.transforms.Top;
-import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 2c3c857..cfef311 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
@@ -15,36 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+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;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Combine;
+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.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
 
 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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Strings;
 
 import org.apache.avro.reflect.Nullable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b1c72e6..c9bada8 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
@@ -15,35 +15,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+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;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
 
 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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
index e5fd015..47631aa 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Count;
+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.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
index 93304eb..1577b77 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
@@ -15,27 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+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.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9dddb5d..021fc14 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
 
 import static com.google.api.services.datastore.client.DatastoreHelper.getPropertyMap;
 import static com.google.api.services.datastore.client.DatastoreHelper.getString;
@@ -23,25 +23,26 @@ import static com.google.api.services.datastore.client.DatastoreHelper.makeFilte
 import static com.google.api.services.datastore.client.DatastoreHelper.makeKey;
 import static com.google.api.services.datastore.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.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+
 import com.google.api.services.datastore.DatastoreV1.Entity;
 import com.google.api.services.datastore.DatastoreV1.Key;
 import com.google.api.services.datastore.DatastoreV1.Property;
 import com.google.api.services.datastore.DatastoreV1.PropertyFilter;
 import com.google.api.services.datastore.DatastoreV1.Query;
 import com.google.api.services.datastore.DatastoreV1.Value;
-import com.google.cloud.dataflow.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
 
 import java.util.Map;
 import java.util.UUID;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 40d1f76..1d34245 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
@@ -15,26 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
 /**
  * This example uses as input Shakespeare's plays as plaintext files, and will remove any
  * duplicate lines across all the files. (The output does not preserve any input order).
  *
  * <p>Concepts: the RemoveDuplicates transform, and how to wire transforms together.
- * Demonstrates {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}/
- * {@link RemoveDuplicates}/{@link com.google.cloud.dataflow.sdk.io.TextIO.Write}.
+ * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/
+ * {@link RemoveDuplicates}/{@link org.apache.beam.sdk.io.TextIO.Write}.
  *
  * <p>To execute this pipeline locally, specify general pipeline configuration:
  *   --project=YOUR_PROJECT_ID

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
index 269ee6a..597ef70 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
@@ -15,25 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Mean;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
index 765420e..7cf138f 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+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.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 
 /**
  * This example shows how to do a join on two collections.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
index b35a57f..3bc1208 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 e32596d..72976e3 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
@@ -15,42 +15,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+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.ExamplePubsubTopicOptions;
+import org.apache.beam.examples.common.PubsubFileInjector;
+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.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.IntraBundleParallelization;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+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.PCollectionList;
 
 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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.examples.common.PubsubFileInjector;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -64,8 +65,9 @@ import java.util.concurrent.TimeUnit;
  * trigger definitions to produce partial (speculative) results before all the data is processed and
  * to control when updated results are produced for late data. The example performs a streaming
  * analysis of the data coming in from PubSub and writes the results to BigQuery. It divides the
- * data into {@link Window windows} to be processed, and demonstrates using various kinds of {@link
- * Trigger triggers} to control when the results for each window are emitted.
+ * data into {@link Window windows} to be processed, and demonstrates using various kinds of
+ * {@link org.apache.beam.sdk.transforms.windowing.Trigger triggers} to control when the results for
+ * each window are emitted.
  *
  * <p> This example uses a portion of real traffic data from San Diego freeways. It contains
  * readings from sensor stations set up along each freeway. Each sensor reading includes a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
index f9e4a2d..f463b1e 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
 
 import com.google.common.io.Files;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
index b1f4f27..c114d6e 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
 
-import com.google.cloud.dataflow.examples.WordCount.CountWords;
-import com.google.cloud.dataflow.examples.WordCount.ExtractWordsFn;
-import com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.examples.WordCount.CountWords;
+import org.apache.beam.examples.WordCount.ExtractWordsFn;
+import org.apache.beam.examples.WordCount.FormatAsTextFn;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
index a70aee1..93dd0be 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
@@ -15,24 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
-
-import com.google.cloud.dataflow.examples.complete.AutoComplete.CompletionCandidate;
-import com.google.cloud.dataflow.examples.complete.AutoComplete.ComputeTopCompletions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+package org.apache.beam.examples.complete;
+
+import org.apache.beam.examples.complete.AutoComplete.CompletionCandidate;
+import org.apache.beam.examples.complete.AutoComplete.ComputeTopCompletions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+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.windowing.SlidingWindows;
+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.TimestampedValue;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
index 5989ce8..c7ce67e 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+package org.apache.beam.examples.complete;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringDelegateCoder;
+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.Create;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
index 52f69b0..d19998e 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete;
+package org.apache.beam.examples.complete;
+
+import org.apache.beam.sdk.Pipeline;
+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.Create;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
index aadfa51..36be568 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn;
+import org.apache.beam.examples.cookbook.BigQueryTornadoes.FormatCountsFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.values.KV;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn;
-import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.FormatCountsFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
index 56efe76..bb475c4 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn;
+import org.apache.beam.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.values.KV;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn;
-import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
index 6e9e3ed..20e2470 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
index 2bd94d5..26393df 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.FilterExamples.FilterSingleMonthDataFn;
+import org.apache.beam.examples.cookbook.FilterExamples.ProjectionFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.FilterExamples.FilterSingleMonthDataFn;
-import com.google.cloud.dataflow.examples.cookbook.FilterExamples.ProjectionFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
index 259ce08..c7827d1 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.JoinExamples.ExtractCountryInfoFn;
+import org.apache.beam.examples.cookbook.JoinExamples.ExtractEventDataFn;
+import org.apache.beam.sdk.Pipeline;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractCountryInfoFn;
-import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractEventDataFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
index f3ffcea..0a84621 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.MaxPerKeyExamples.ExtractTempFn;
+import org.apache.beam.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.values.KV;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.ExtractTempFn;
-import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.CoreMatchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
index 048d803..4f15aef 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.cookbook;
+package org.apache.beam.examples.cookbook;
+
+import org.apache.beam.examples.cookbook.TriggerExample.ExtractFlowInfo;
+import org.apache.beam.examples.cookbook.TriggerExample.TotalFlow;
+import org.apache.beam.sdk.Pipeline;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+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.TimestampedValue;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.TriggerExample.ExtractFlowInfo;
-import com.google.cloud.dataflow.examples.cookbook.TriggerExample.TotalFlow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 8705ed3..493344e 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
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples;
+package org.apache.beam.examples;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.FlatMapElements;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import java.util.Arrays;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 89832b2..d93c2ae 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
@@ -15,37 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.Mean;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.common.DataflowExampleUtils;
+import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+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.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+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.TypeDescriptor;
 
 import org.joda.time.DateTimeZone;
 import org.joda.time.Duration;
@@ -76,7 +76,7 @@ import java.util.TimeZone;
  * subsequent fixed windowing. (This could be used to tell us what games are giving us greater
  * user retention).
  *
- * <p> Run {@code com.google.cloud.dataflow.examples.complete.game.injector.Injector} to generate
+ * <p> Run {@code org.apache.beam.examples.complete.game.injector.Injector} to generate
  * pubsub data for this pipeline. The {@code Injector} documentation provides more detail.
  *
  * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 58944c5..5ce7d95 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
@@ -15,22 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.WithTimestamps;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
+package org.apache.beam.examples.complete.game;
+
+import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.sdk.Pipeline;
+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;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
 
 import org.joda.time.DateTimeZone;
 import org.joda.time.Duration;



[37/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java
deleted file mode 100644
index 632be6d..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java
+++ /dev/null
@@ -1,394 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.NanoClock;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowMetricUpdateExtractor;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A DataflowPipelineJob represents a job submitted to Dataflow using
- * {@link DataflowPipelineRunner}.
- */
-public class DataflowPipelineJob implements PipelineResult {
-  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);
-
-  /**
-   * The id for the job.
-   */
-  private String jobId;
-
-  /**
-   * Google cloud project to associate this pipeline with.
-   */
-  private String projectId;
-
-  /**
-   * Client for the Dataflow service. This can be used to query the service
-   * for information about the job.
-   */
-  private Dataflow dataflowClient;
-
-  /**
-   * The state the job terminated in or {@code null} if the job has not terminated.
-   */
-  @Nullable
-  private State terminalState = null;
-
-  /**
-   * The job that replaced this one or {@code null} if the job has not been replaced.
-   */
-  @Nullable
-  private DataflowPipelineJob replacedByJob = null;
-
-  private DataflowAggregatorTransforms aggregatorTransforms;
-
-  /**
-   * The Metric Updates retrieved after the job was in a terminal state.
-   */
-  private List<MetricUpdate> terminalMetricUpdates;
-
-  /**
-   * The polling interval for job status and messages information.
-   */
-  static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
-  static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
-
-  /**
-   * The amount of polling attempts for job status and messages information.
-   */
-  static final int MESSAGES_POLLING_ATTEMPTS = 10;
-  static final int STATUS_POLLING_ATTEMPTS = 5;
-
-  /**
-   * Constructs the job.
-   *
-   * @param projectId the project id
-   * @param jobId the job id
-   * @param dataflowClient the client for the Dataflow Service
-   */
-  public DataflowPipelineJob(String projectId, String jobId, Dataflow dataflowClient,
-      DataflowAggregatorTransforms aggregatorTransforms) {
-    this.projectId = projectId;
-    this.jobId = jobId;
-    this.dataflowClient = dataflowClient;
-    this.aggregatorTransforms = aggregatorTransforms;
-  }
-
-  /**
-   * Get the id of this job.
-   */
-  public String getJobId() {
-    return jobId;
-  }
-
-  /**
-   * Get the project this job exists in.
-   */
-  public String getProjectId() {
-    return projectId;
-  }
-
-  /**
-   * Returns a new {@link DataflowPipelineJob} for the job that replaced this one, if applicable.
-   *
-   * @throws IllegalStateException if called before the job has terminated or if the job terminated
-   * but was not updated
-   */
-  public DataflowPipelineJob getReplacedByJob() {
-    if (terminalState == null) {
-      throw new IllegalStateException("getReplacedByJob() called before job terminated");
-    }
-    if (replacedByJob == null) {
-      throw new IllegalStateException("getReplacedByJob() called for job that was not replaced");
-    }
-    return replacedByJob;
-  }
-
-  /**
-   * Get the Cloud Dataflow API Client used by this job.
-   */
-  public Dataflow getDataflowClient() {
-    return dataflowClient;
-  }
-
-  /**
-   * Waits for the job to finish and return the final status.
-   *
-   * @param timeToWait The time to wait in units timeUnit for the job to finish.
-   *     Provide a value less than 1 ms for an infinite wait.
-   * @param timeUnit The unit of time for timeToWait.
-   * @param messageHandler If non null this handler will be invoked for each
-   *   batch of messages received.
-   * @return The final state of the job or null on timeout or if the
-   *   thread is interrupted.
-   * @throws IOException If there is a persistent problem getting job
-   *   information.
-   * @throws InterruptedException
-   */
-  @Nullable
-  public State waitToFinish(
-      long timeToWait,
-      TimeUnit timeUnit,
-      MonitoringUtil.JobMessagesHandler messageHandler)
-          throws IOException, InterruptedException {
-    return waitToFinish(timeToWait, timeUnit, messageHandler, Sleeper.DEFAULT, NanoClock.SYSTEM);
-  }
-
-  /**
-   * Wait for the job to finish and return the final status.
-   *
-   * @param timeToWait The time to wait in units timeUnit for the job to finish.
-   *     Provide a value less than 1 ms for an infinite wait.
-   * @param timeUnit The unit of time for timeToWait.
-   * @param messageHandler If non null this handler will be invoked for each
-   *   batch of messages received.
-   * @param sleeper A sleeper to use to sleep between attempts.
-   * @param nanoClock A nanoClock used to time the total time taken.
-   * @return The final state of the job or null on timeout or if the
-   *   thread is interrupted.
-   * @throws IOException If there is a persistent problem getting job
-   *   information.
-   * @throws InterruptedException
-   */
-  @Nullable
-  @VisibleForTesting
-  State waitToFinish(
-      long timeToWait,
-      TimeUnit timeUnit,
-      MonitoringUtil.JobMessagesHandler messageHandler,
-      Sleeper sleeper,
-      NanoClock nanoClock)
-          throws IOException, InterruptedException {
-    MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowClient);
-
-    long lastTimestamp = 0;
-    BackOff backoff =
-        timeUnit.toMillis(timeToWait) > 0
-            ? new AttemptAndTimeBoundedExponentialBackOff(
-                MESSAGES_POLLING_ATTEMPTS,
-                MESSAGES_POLLING_INTERVAL,
-                timeUnit.toMillis(timeToWait),
-                AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
-                nanoClock)
-            : new AttemptBoundedExponentialBackOff(
-                MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL);
-    State state;
-    do {
-      // Get the state of the job before listing messages. This ensures we always fetch job
-      // messages after the job finishes to ensure we have all them.
-      state = getStateWithRetries(1, sleeper);
-      boolean hasError = state == State.UNKNOWN;
-
-      if (messageHandler != null && !hasError) {
-        // Process all the job messages that have accumulated so far.
-        try {
-          List<JobMessage> allMessages = monitor.getJobMessages(
-              jobId, lastTimestamp);
-
-          if (!allMessages.isEmpty()) {
-            lastTimestamp =
-                fromCloudTime(allMessages.get(allMessages.size() - 1).getTime()).getMillis();
-            messageHandler.process(allMessages);
-          }
-        } catch (GoogleJsonResponseException | SocketTimeoutException e) {
-          hasError = true;
-          LOG.warn("There were problems getting current job messages: {}.", e.getMessage());
-          LOG.debug("Exception information:", e);
-        }
-      }
-
-      if (!hasError) {
-        backoff.reset();
-        // Check if the job is done.
-        if (state.isTerminal()) {
-          return state;
-        }
-      }
-    } while(BackOffUtils.next(sleeper, backoff));
-    LOG.warn("No terminal state was returned.  State value {}", state);
-    return null;  // Timed out.
-  }
-
-  /**
-   * Cancels the job.
-   * @throws IOException if there is a problem executing the cancel request.
-   */
-  public void cancel() throws IOException {
-    Job content = new Job();
-    content.setProjectId(projectId);
-    content.setId(jobId);
-    content.setRequestedState("JOB_STATE_CANCELLED");
-    dataflowClient.projects().jobs()
-        .update(projectId, jobId, content)
-        .execute();
-  }
-
-  @Override
-  public State getState() {
-    if (terminalState != null) {
-      return terminalState;
-    }
-
-    return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT);
-  }
-
-  /**
-   * Attempts to get the state. Uses exponential backoff on failure up to the maximum number
-   * of passed in attempts.
-   *
-   * @param attempts The amount of attempts to make.
-   * @param sleeper Object used to do the sleeps between attempts.
-   * @return The state of the job or State.UNKNOWN in case of failure.
-   */
-  @VisibleForTesting
-  State getStateWithRetries(int attempts, Sleeper sleeper) {
-    if (terminalState != null) {
-      return terminalState;
-    }
-    try {
-      Job job = getJobWithRetries(attempts, sleeper);
-      return MonitoringUtil.toState(job.getCurrentState());
-    } catch (IOException exn) {
-      // The only IOException that getJobWithRetries is permitted to throw is the final IOException
-      // that caused the failure of retry. Other exceptions are wrapped in an unchecked exceptions
-      // and will propagate.
-      return State.UNKNOWN;
-    }
-  }
-
-  /**
-   * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the
-   * maximum number of passed in attempts.
-   *
-   * @param attempts The amount of attempts to make.
-   * @param sleeper Object used to do the sleeps between attempts.
-   * @return The underlying {@link Job} object.
-   * @throws IOException When the maximum number of retries is exhausted, the last exception is
-   * thrown.
-   */
-  @VisibleForTesting
-  Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException {
-    AttemptBoundedExponentialBackOff backoff =
-        new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL);
-
-    // Retry loop ends in return or throw
-    while (true) {
-      try {
-        Job job = dataflowClient
-            .projects()
-            .jobs()
-            .get(projectId, jobId)
-            .execute();
-        State currentState = MonitoringUtil.toState(job.getCurrentState());
-        if (currentState.isTerminal()) {
-          terminalState = currentState;
-          replacedByJob = new DataflowPipelineJob(
-              getProjectId(), job.getReplacedByJobId(), dataflowClient, aggregatorTransforms);
-        }
-        return job;
-      } catch (IOException exn) {
-        LOG.warn("There were problems getting current job status: {}.", exn.getMessage());
-        LOG.debug("Exception information:", exn);
-
-        if (!nextBackOff(sleeper, backoff)) {
-          throw exn;
-        }
-      }
-    }
-  }
-
-  /**
-   * Identical to {@link BackOffUtils#next} but without checked exceptions.
-   */
-  private boolean nextBackOff(Sleeper sleeper, BackOff backoff) {
-    try {
-      return BackOffUtils.next(sleeper, backoff);
-    } catch (InterruptedException | IOException e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw Throwables.propagate(e);
-    }
-  }
-
-  @Override
-  public <OutputT> AggregatorValues<OutputT> getAggregatorValues(Aggregator<?, OutputT> aggregator)
-      throws AggregatorRetrievalException {
-    try {
-      return new MapAggregatorValues<>(fromMetricUpdates(aggregator));
-    } catch (IOException e) {
-      throw new AggregatorRetrievalException(
-          "IOException when retrieving Aggregator values for Aggregator " + aggregator, e);
-    }
-  }
-
-  private <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator)
-      throws IOException {
-    if (aggregatorTransforms.contains(aggregator)) {
-      List<MetricUpdate> metricUpdates;
-      if (terminalMetricUpdates != null) {
-        metricUpdates = terminalMetricUpdates;
-      } else {
-        boolean terminal = getState().isTerminal();
-        JobMetrics jobMetrics =
-            dataflowClient.projects().jobs().getMetrics(projectId, jobId).execute();
-        metricUpdates = jobMetrics.getMetrics();
-        if (terminal && jobMetrics.getMetrics() != null) {
-          terminalMetricUpdates = metricUpdates;
-        }
-      }
-
-      return DataflowMetricUpdateExtractor.fromMetricUpdates(
-          aggregator, aggregatorTransforms, metricUpdates);
-    } else {
-      throw new IllegalArgumentException(
-          "Aggregator " + aggregator + " is not used in this pipeline");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java
deleted file mode 100644
index 8aaa7cc..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java
+++ /dev/null
@@ -1,59 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for
- * the {@link DataflowPipeline}.
- */
-public class DataflowPipelineRegistrar {
-  private DataflowPipelineRegistrar() { }
-
-  /**
-   * Register the {@link DataflowPipelineOptions} and {@link BlockingDataflowPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(
-          DataflowPipelineOptions.class,
-          BlockingDataflowPipelineOptions.class);
-    }
-  }
-
-  /**
-   * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class Runner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
-          DataflowPipelineRunner.class,
-          BlockingDataflowPipelineRunner.class);
-    }
-  }
-}


[30/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
new file mode 100644
index 0000000..e961066
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
@@ -0,0 +1,256 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.testing;
+
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.DataflowJobExecutionException;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.common.base.Optional;
+import com.google.common.base.Throwables;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
+ * {@link DataflowPipelineRunner} when running tests against the {@link TestPipeline}.
+ *
+ * @see TestPipeline
+ */
+public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
+  private static final String TENTATIVE_COUNTER = "tentative";
+  private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
+
+  private final TestDataflowPipelineOptions options;
+  private final DataflowPipelineRunner runner;
+  private int expectedNumberOfAssertions = 0;
+
+  TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
+    this.options = options;
+    this.runner = DataflowPipelineRunner.fromOptions(options);
+  }
+
+  /**
+   * Constructs a runner from the provided options.
+   */
+  public static TestDataflowPipelineRunner fromOptions(
+      PipelineOptions options) {
+    TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
+
+    return new TestDataflowPipelineRunner(dataflowOptions);
+  }
+
+  @Override
+  public DataflowPipelineJob run(Pipeline pipeline) {
+    return run(pipeline, runner);
+  }
+
+  DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
+
+    final DataflowPipelineJob job;
+    try {
+      job = runner.run(pipeline);
+    } catch (DataflowJobExecutionException ex) {
+      throw new IllegalStateException("The dataflow failed.");
+    }
+
+    LOG.info("Running Dataflow job {} with {} expected assertions.",
+        job.getJobId(), expectedNumberOfAssertions);
+
+    CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
+        job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+
+    try {
+      final Optional<Boolean> result;
+
+      if (options.isStreaming()) {
+        Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
+            new Callable<Optional<Boolean>>() {
+          @Override
+          public Optional<Boolean> call() throws Exception {
+            try {
+              for (;;) {
+                Optional<Boolean> result = checkForSuccess(job);
+                if (result.isPresent()) {
+                  return result;
+                }
+                Thread.sleep(10000L);
+              }
+            } finally {
+              LOG.info("Cancelling Dataflow job {}", job.getJobId());
+              job.cancel();
+            }
+          }
+        });
+        State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
+        if (finalState == null || finalState == State.RUNNING) {
+          LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
+              job.getJobId());
+          job.cancel();
+        }
+        result = resultFuture.get();
+      } else {
+        job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
+        result = checkForSuccess(job);
+      }
+      if (!result.isPresent()) {
+        throw new IllegalStateException(
+            "The dataflow did not output a success or failure metric.");
+      } else if (!result.get()) {
+        throw new AssertionError(messageHandler.getErrorMessage() == null ?
+            "The dataflow did not return a failure reason."
+            : messageHandler.getErrorMessage());
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause());
+      throw new RuntimeException(e.getCause());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return job;
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    if (transform instanceof PAssert.OneSideInputAssert
+        || transform instanceof PAssert.TwoSideInputAssert) {
+      expectedNumberOfAssertions += 1;
+    }
+
+    return runner.apply(transform, input);
+  }
+
+  Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
+      throws IOException {
+    State state = job.getState();
+    if (state == State.FAILED || state == State.CANCELLED) {
+      LOG.info("The pipeline failed");
+      return Optional.of(false);
+    }
+
+    JobMetrics metrics = job.getDataflowClient().projects().jobs()
+        .getMetrics(job.getProjectId(), job.getJobId()).execute();
+
+    if (metrics == null || metrics.getMetrics() == null) {
+      LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
+    } else {
+      int successes = 0;
+      int failures = 0;
+      for (MetricUpdate metric : metrics.getMetrics()) {
+        if (metric.getName() == null || metric.getName().getContext() == null
+            || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
+          // Don't double count using the non-tentative version of the metric.
+          continue;
+        }
+        if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
+          successes += ((BigDecimal) metric.getScalar()).intValue();
+        } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
+          failures += ((BigDecimal) metric.getScalar()).intValue();
+        }
+      }
+
+      if (failures > 0) {
+        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+            + "{} expected assertions.", job.getJobId(), successes, failures,
+            expectedNumberOfAssertions);
+        return Optional.of(false);
+      } else if (successes >= expectedNumberOfAssertions) {
+        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+            + "{} expected assertions.", job.getJobId(), successes, failures,
+            expectedNumberOfAssertions);
+        return Optional.of(true);
+      }
+
+      LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
+          + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
+    }
+
+    return Optional.<Boolean>absent();
+  }
+
+  @Override
+  public String toString() {
+    return "TestDataflowPipelineRunner#" + options.getAppName();
+  }
+
+  /**
+   * Cancels the workflow on the first error message it sees.
+   *
+   * <p>Creates an error message representing the concatenation of all error messages seen.
+   */
+  private static class CancelWorkflowOnError implements JobMessagesHandler {
+    private final DataflowPipelineJob job;
+    private final JobMessagesHandler messageHandler;
+    private final StringBuffer errorMessage;
+    private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
+      this.job = job;
+      this.messageHandler = messageHandler;
+      this.errorMessage = new StringBuffer();
+    }
+
+    @Override
+    public void process(List<JobMessage> messages) {
+      messageHandler.process(messages);
+      for (JobMessage message : messages) {
+        if (message.getMessageImportance() != null
+            && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
+          LOG.info("Dataflow job {} threw exception. Failure message was: {}",
+              job.getJobId(), message.getMessageText());
+          errorMessage.append(message.getMessageText());
+        }
+      }
+      if (errorMessage.length() > 0) {
+        LOG.info("Cancelling Dataflow job {}", job.getJobId());
+        try {
+          job.cancel();
+        } catch (Exception ignore) {
+          // The TestDataflowPipelineRunner will thrown an AssertionError with the job failure
+          // messages.
+        }
+      }
+    }
+
+    private String getErrorMessage() {
+      return errorMessage.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
new file mode 100644
index 0000000..aeb864a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
@@ -0,0 +1,98 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
+ */
+public class DataflowPathValidator implements PathValidator {
+
+  private DataflowPipelineOptions dataflowOptions;
+
+  DataflowPathValidator(DataflowPipelineOptions options) {
+    this.dataflowOptions = options;
+  }
+
+  public static DataflowPathValidator fromOptions(PipelineOptions options) {
+    return new DataflowPathValidator(options.as(DataflowPipelineOptions.class));
+  }
+
+  /**
+   * Validates the the input GCS path is accessible and that the path
+   * is well formed.
+   */
+  @Override
+  public String validateInputFilePatternSupported(String filepattern) {
+    GcsPath gcsPath = getGcsPath(filepattern);
+    Preconditions.checkArgument(
+        dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
+    String returnValue = verifyPath(filepattern);
+    verifyPathIsAccessible(filepattern, "Could not find file %s");
+    return returnValue;
+  }
+
+  /**
+   * Validates the the output GCS path is accessible and that the path
+   * is well formed.
+   */
+  @Override
+  public String validateOutputFilePrefixSupported(String filePrefix) {
+    String returnValue = verifyPath(filePrefix);
+    verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
+    return returnValue;
+  }
+
+  @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("//"),
+        "Dataflow Service does not allow objects with consecutive slashes");
+    return gcsPath.toResourceName();
+  }
+
+  private void verifyPathIsAccessible(String path, String errorMessage) {
+    GcsPath gcsPath = getGcsPath(path);
+    try {
+      Preconditions.checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
+        errorMessage, path);
+    } catch (IOException e) {
+      throw new RuntimeException(
+          String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
+          e);
+    }
+  }
+
+  private GcsPath getGcsPath(String path) {
+    try {
+      return GcsPath.fromUri(path);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(String.format(
+          "%s expected a valid 'gs://' path but was given '%s'",
+          dataflowOptions.getRunner().getSimpleName(), path), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
new file mode 100644
index 0000000..18e6654
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
@@ -0,0 +1,112 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import static com.google.cloud.dataflow.sdk.util.Transport.getJsonFactory;
+import static com.google.cloud.dataflow.sdk.util.Transport.getTransport;
+
+import com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.services.clouddebugger.v2.Clouddebugger;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
+import com.google.common.collect.ImmutableList;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+
+/**
+ * Helpers for cloud communication.
+ */
+public class DataflowTransport {
+
+
+  private static class ApiComponents {
+    public String rootUrl;
+    public String servicePath;
+
+    public ApiComponents(String root, String path) {
+      this.rootUrl = root;
+      this.servicePath = path;
+    }
+  }
+
+  private static ApiComponents apiComponentsFromUrl(String urlString) {
+    try {
+      URL url = new URL(urlString);
+      String rootUrl = url.getProtocol() + "://" + url.getHost() +
+          (url.getPort() > 0 ? ":" + url.getPort() : "");
+      return new ApiComponents(rootUrl, url.getPath());
+    } catch (MalformedURLException e) {
+      throw new RuntimeException("Invalid URL: " + urlString);
+    }
+  }
+
+  /**
+   * Returns a Google Cloud Dataflow client builder.
+   */
+  public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options) {
+    String servicePath = options.getDataflowEndpoint();
+    ApiComponents components;
+    if (servicePath.contains("://")) {
+      components = apiComponentsFromUrl(servicePath);
+    } else {
+      components = new ApiComponents(options.getApiRootUrl(), servicePath);
+    }
+
+    return new Dataflow.Builder(getTransport(),
+        getJsonFactory(),
+        chainHttpRequestInitializer(
+            options.getGcpCredential(),
+            // Do not log 404. It clutters the output and is possibly even required by the caller.
+            new RetryHttpRequestInitializer(ImmutableList.of(404))))
+        .setApplicationName(options.getAppName())
+        .setRootUrl(components.rootUrl)
+        .setServicePath(components.servicePath)
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  public static Clouddebugger.Builder newClouddebuggerClient(DataflowPipelineOptions options) {
+    return new Clouddebugger.Builder(getTransport(),
+        getJsonFactory(),
+        chainHttpRequestInitializer(options.getGcpCredential(), new RetryHttpRequestInitializer()))
+        .setApplicationName(options.getAppName())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  /**
+   * Returns a Dataflow client that does not automatically retry failed
+   * requests.
+   */
+  public static Dataflow.Builder
+      newRawDataflowClient(DataflowPipelineOptions options) {
+    return newDataflowClient(options)
+        .setHttpRequestInitializer(options.getGcpCredential())
+        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+  }
+
+  private static HttpRequestInitializer chainHttpRequestInitializer(
+      Credential credential, HttpRequestInitializer httpRequestInitializer) {
+    if (credential == null) {
+      return httpRequestInitializer;
+    } else {
+      return new ChainingHttpRequestInitializer(credential, httpRequestInitializer);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
new file mode 100644
index 0000000..7307e83
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
@@ -0,0 +1,54 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * Utility class for staging files to GCS.
+ */
+public class GcsStager implements Stager {
+  private DataflowPipelineOptions options;
+
+  private GcsStager(DataflowPipelineOptions options) {
+    this.options = options;
+  }
+
+  public static GcsStager fromOptions(PipelineOptions options) {
+    return new GcsStager(options.as(DataflowPipelineOptions.class));
+  }
+
+  @Override
+  public List<DataflowPackage> stageFiles() {
+    Preconditions.checkNotNull(options.getStagingLocation());
+    List<String> filesToStage = options.getFilesToStage();
+    String windmillBinary =
+        options.as(DataflowPipelineDebugOptions.class).getOverrideWindmillBinary();
+    if (windmillBinary != null) {
+      filesToStage.add("windmill_main=" + windmillBinary);
+    }
+    return PackageUtil.stageClasspathElements(
+        options.getFilesToStage(), options.getStagingLocation());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
new file mode 100644
index 0000000..2c06a92
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
@@ -0,0 +1,235 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.ListJobMessagesResponse;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * A helper class for monitoring jobs submitted to the service.
+ */
+public final class MonitoringUtil {
+
+  private static final String GCLOUD_DATAFLOW_PREFIX = "gcloud alpha dataflow";
+  private static final String ENDPOINT_OVERRIDE_ENV_VAR =
+      "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW";
+
+  private static final Map<String, State> DATAFLOW_STATE_TO_JOB_STATE =
+      ImmutableMap
+          .<String, State>builder()
+          .put("JOB_STATE_UNKNOWN", State.UNKNOWN)
+          .put("JOB_STATE_STOPPED", State.STOPPED)
+          .put("JOB_STATE_RUNNING", State.RUNNING)
+          .put("JOB_STATE_DONE", State.DONE)
+          .put("JOB_STATE_FAILED", State.FAILED)
+          .put("JOB_STATE_CANCELLED", State.CANCELLED)
+          .put("JOB_STATE_UPDATED", State.UPDATED)
+          .build();
+
+  private String projectId;
+  private Messages messagesClient;
+
+  /**
+   * An interface that can be used for defining callbacks to receive a list
+   * of JobMessages containing monitoring information.
+   */
+  public interface JobMessagesHandler {
+    /** Process the rows. */
+    void process(List<JobMessage> messages);
+  }
+
+  /** A handler that prints monitoring messages to a stream. */
+  public static class PrintHandler implements JobMessagesHandler {
+    private PrintStream out;
+
+    /**
+     * Construct the handler.
+     *
+     * @param stream The stream to write the messages to.
+     */
+    public PrintHandler(PrintStream stream) {
+      out = stream;
+    }
+
+    @Override
+    public void process(List<JobMessage> messages) {
+      for (JobMessage message : messages) {
+        if (message.getMessageText() == null || message.getMessageText().isEmpty()) {
+          continue;
+        }
+        String importanceString = null;
+        if (message.getMessageImportance() == null) {
+          continue;
+        } else if (message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
+          importanceString = "Error:   ";
+        } else if (message.getMessageImportance().equals("JOB_MESSAGE_WARNING")) {
+          importanceString = "Warning: ";
+        } else if (message.getMessageImportance().equals("JOB_MESSAGE_BASIC")) {
+          importanceString = "Basic:  ";
+        } else if (message.getMessageImportance().equals("JOB_MESSAGE_DETAILED")) {
+          importanceString = "Detail:  ";
+        } else {
+          // TODO: Remove filtering here once getJobMessages supports minimum
+          // importance.
+          continue;
+        }
+        @Nullable Instant time = TimeUtil.fromCloudTime(message.getTime());
+        if (time == null) {
+          out.print("UNKNOWN TIMESTAMP: ");
+        } else {
+          out.print(time + ": ");
+        }
+        if (importanceString != null) {
+          out.print(importanceString);
+        }
+        out.println(message.getMessageText());
+      }
+      out.flush();
+    }
+  }
+
+  /** Construct a helper for monitoring. */
+  public MonitoringUtil(String projectId, Dataflow dataflow) {
+    this(projectId, dataflow.projects().jobs().messages());
+  }
+
+  // @VisibleForTesting
+  MonitoringUtil(String projectId, Messages messagesClient) {
+    this.projectId = projectId;
+    this.messagesClient = messagesClient;
+  }
+
+  /**
+   * Comparator for sorting rows in increasing order based on timestamp.
+   */
+  public static class TimeStampComparator implements Comparator<JobMessage> {
+    @Override
+    public int compare(JobMessage o1, JobMessage o2) {
+      @Nullable Instant t1 = fromCloudTime(o1.getTime());
+      if (t1 == null) {
+        return -1;
+      }
+      @Nullable Instant t2 = fromCloudTime(o2.getTime());
+      if (t2 == null) {
+        return 1;
+      }
+      return t1.compareTo(t2);
+    }
+  }
+
+  /**
+   * Return job messages sorted in ascending order by timestamp.
+   * @param jobId The id of the job to get the messages for.
+   * @param startTimestampMs Return only those messages with a
+   *   timestamp greater than this value.
+   * @return collection of messages
+   * @throws IOException
+   */
+  public ArrayList<JobMessage> getJobMessages(
+      String jobId, long startTimestampMs) throws IOException {
+    // TODO: Allow filtering messages by importance
+    Instant startTimestamp = new Instant(startTimestampMs);
+    ArrayList<JobMessage> allMessages = new ArrayList<>();
+    String pageToken = null;
+    while (true) {
+      Messages.List listRequest = messagesClient.list(projectId, jobId);
+      if (pageToken != null) {
+        listRequest.setPageToken(pageToken);
+      }
+      ListJobMessagesResponse response = listRequest.execute();
+
+      if (response == null || response.getJobMessages() == null) {
+        return allMessages;
+      }
+
+      for (JobMessage m : response.getJobMessages()) {
+        @Nullable Instant timestamp = fromCloudTime(m.getTime());
+        if (timestamp == null) {
+          continue;
+        }
+        if (timestamp.isAfter(startTimestamp)) {
+          allMessages.add(m);
+        }
+      }
+
+      if (response.getNextPageToken() == null) {
+        break;
+      } else {
+        pageToken = response.getNextPageToken();
+      }
+    }
+
+    Collections.sort(allMessages, new TimeStampComparator());
+    return allMessages;
+  }
+
+  public static String getJobMonitoringPageURL(String projectName, String jobId) {
+    try {
+      // Project name is allowed in place of the project id: the user will be redirected to a URL
+      // that has the project name replaced with project id.
+      return String.format(
+          "https://console.developers.google.com/project/%s/dataflow/job/%s",
+          URLEncoder.encode(projectName, "UTF-8"),
+          URLEncoder.encode(jobId, "UTF-8"));
+    } catch (UnsupportedEncodingException e) {
+      // Should never happen.
+      throw new AssertionError("UTF-8 encoding is not supported by the environment", e);
+    }
+  }
+
+  public static String getGcloudCancelCommand(DataflowPipelineOptions options, String jobId) {
+
+    // If using a different Dataflow API than default, prefix command with an API override.
+    String dataflowApiOverridePrefix = "";
+    String apiUrl = options.getDataflowClient().getBaseUrl();
+    if (!apiUrl.equals(Dataflow.DEFAULT_BASE_URL)) {
+      dataflowApiOverridePrefix = String.format("%s=%s ", ENDPOINT_OVERRIDE_ENV_VAR, apiUrl);
+    }
+
+    // Assemble cancel command from optional prefix and project/job parameters.
+    return String.format("%s%s jobs --project=%s cancel %s",
+        dataflowApiOverridePrefix, GCLOUD_DATAFLOW_PREFIX, options.getProject(), jobId);
+  }
+
+  public static State toState(String stateName) {
+    return MoreObjects.firstNonNull(DATAFLOW_STATE_TO_JOB_STATE.get(stateName),
+        State.UNKNOWN);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
new file mode 100644
index 0000000..0e234a8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
@@ -0,0 +1,328 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.hash.Funnels;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import com.google.common.io.CountingOutputStream;
+import com.google.common.io.Files;
+
+import com.fasterxml.jackson.core.Base64Variants;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+
+/** Helper routines for packages. */
+public class PackageUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class);
+  /**
+   * A reasonable upper bound on the number of jars required to launch a Dataflow job.
+   */
+  public static final int SANE_CLASSPATH_SIZE = 1000;
+  /**
+   * The initial interval to use between package staging attempts.
+   */
+  private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L;
+  /**
+   * The maximum number of attempts when staging a file.
+   */
+  private static final int MAX_ATTEMPTS = 5;
+
+  /**
+   * Translates exceptions from API calls.
+   */
+  private static final ApiErrorExtractor ERROR_EXTRACTOR = new ApiErrorExtractor();
+
+  /**
+   * Creates a DataflowPackage containing information about how a classpath element should be
+   * staged, including the staging destination as well as its size and hash.
+   *
+   * @param classpathElement The local path for the classpath element.
+   * @param stagingPath The base location for staged classpath elements.
+   * @param overridePackageName If non-null, use the given value as the package name
+   *                            instead of generating one automatically.
+   * @return The package.
+   */
+  @Deprecated
+  public static DataflowPackage createPackage(File classpathElement,
+      String stagingPath, String overridePackageName) {
+    return createPackageAttributes(classpathElement, stagingPath, overridePackageName)
+        .getDataflowPackage();
+  }
+
+  /**
+   * Compute and cache the attributes of a classpath element that we will need to stage it.
+   *
+   * @param classpathElement the file or directory to be staged.
+   * @param stagingPath The base location for staged classpath elements.
+   * @param overridePackageName If non-null, use the given value as the package name
+   *                            instead of generating one automatically.
+   * @return a {@link PackageAttributes} that containing metadata about the object to be staged.
+   */
+  static PackageAttributes createPackageAttributes(File classpathElement,
+      String stagingPath, String overridePackageName) {
+    try {
+      boolean directory = classpathElement.isDirectory();
+
+      // Compute size and hash in one pass over file or directory.
+      Hasher hasher = Hashing.md5().newHasher();
+      OutputStream hashStream = Funnels.asOutputStream(hasher);
+      CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream);
+
+      if (!directory) {
+        // Files are staged as-is.
+        Files.asByteSource(classpathElement).copyTo(countingOutputStream);
+      } else {
+        // Directories are recursively zipped.
+        ZipFiles.zipDirectory(classpathElement, countingOutputStream);
+      }
+
+      long size = countingOutputStream.getCount();
+      String hash = Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes());
+
+      // Create the DataflowPackage with staging name and location.
+      String uniqueName = getUniqueContentName(classpathElement, hash);
+      String resourcePath = IOChannelUtils.resolve(stagingPath, uniqueName);
+      DataflowPackage target = new DataflowPackage();
+      target.setName(overridePackageName != null ? overridePackageName : uniqueName);
+      target.setLocation(resourcePath);
+
+      return new PackageAttributes(size, hash, directory, target);
+    } catch (IOException e) {
+      throw new RuntimeException("Package setup failure for " + classpathElement, e);
+    }
+  }
+
+  /**
+   * Transfers the classpath elements to the staging location.
+   *
+   * @param classpathElements The elements to stage.
+   * @param stagingPath The base location to stage the elements to.
+   * @return A list of cloud workflow packages, each representing a classpath element.
+   */
+  public static List<DataflowPackage> stageClasspathElements(
+      Collection<String> classpathElements, String stagingPath) {
+    return stageClasspathElements(classpathElements, stagingPath, Sleeper.DEFAULT);
+  }
+
+  // Visible for testing.
+  static List<DataflowPackage> stageClasspathElements(
+      Collection<String> classpathElements, String stagingPath,
+      Sleeper retrySleeper) {
+    LOG.info("Uploading {} files from PipelineOptions.filesToStage to staging location to "
+        + "prepare for execution.", classpathElements.size());
+
+    if (classpathElements.size() > SANE_CLASSPATH_SIZE) {
+      LOG.warn("Your classpath contains {} elements, which Google Cloud Dataflow automatically "
+          + "copies to all workers. Having this many entries on your classpath may be indicative "
+          + "of an issue in your pipeline. You may want to consider trimming the classpath to "
+          + "necessary dependencies only, using --filesToStage pipeline option to override "
+          + "what files are being staged, or bundling several dependencies into one.",
+          classpathElements.size());
+    }
+
+    ArrayList<DataflowPackage> packages = new ArrayList<>();
+
+    if (stagingPath == null) {
+      throw new IllegalArgumentException(
+          "Can't stage classpath elements on because no staging location has been provided");
+    }
+
+    int numUploaded = 0;
+    int numCached = 0;
+    for (String classpathElement : classpathElements) {
+      String packageName = null;
+      if (classpathElement.contains("=")) {
+        String[] components = classpathElement.split("=", 2);
+        packageName = components[0];
+        classpathElement = components[1];
+      }
+
+      File file = new File(classpathElement);
+      if (!file.exists()) {
+        LOG.warn("Skipping non-existent classpath element {} that was specified.",
+            classpathElement);
+        continue;
+      }
+
+      PackageAttributes attributes = createPackageAttributes(file, stagingPath, packageName);
+
+      DataflowPackage workflowPackage = attributes.getDataflowPackage();
+      packages.add(workflowPackage);
+      String target = workflowPackage.getLocation();
+
+      // TODO: Should we attempt to detect the Mime type rather than
+      // always using MimeTypes.BINARY?
+      try {
+        try {
+          long remoteLength = IOChannelUtils.getSizeBytes(target);
+          if (remoteLength == attributes.getSize()) {
+            LOG.debug("Skipping classpath element already staged: {} at {}",
+                classpathElement, target);
+            numCached++;
+            continue;
+          }
+        } catch (FileNotFoundException expected) {
+          // If the file doesn't exist, it means we need to upload it.
+        }
+
+        // Upload file, retrying on failure.
+        AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
+            MAX_ATTEMPTS,
+            INITIAL_BACKOFF_INTERVAL_MS);
+        while (true) {
+          try {
+            LOG.debug("Uploading classpath element {} to {}", classpathElement, target);
+            try (WritableByteChannel writer = IOChannelUtils.create(target, MimeTypes.BINARY)) {
+              copyContent(classpathElement, writer);
+            }
+            numUploaded++;
+            break;
+          } catch (IOException e) {
+            if (ERROR_EXTRACTOR.accessDenied(e)) {
+              String errorMessage = String.format(
+                  "Uploaded failed due to permissions error, will NOT retry staging "
+                  + "of classpath %s. Please verify credentials are valid and that you have "
+                  + "write access to %s. Stale credentials can be resolved by executing "
+                  + "'gcloud auth login'.", classpathElement, target);
+              LOG.error(errorMessage);
+              throw new IOException(errorMessage, e);
+            } else if (!backoff.atMaxAttempts()) {
+              LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
+                  classpathElement, e);
+              BackOffUtils.next(retrySleeper, backoff);
+            } else {
+              // Rethrow last error, to be included as a cause in the catch below.
+              LOG.error("Upload failed, will NOT retry staging of classpath: {}",
+                  classpathElement, e);
+              throw e;
+            }
+          }
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Could not stage classpath element: " + classpathElement, e);
+      }
+    }
+
+    LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, "
+        + "{} files cached",
+        numUploaded, numCached);
+
+    return packages;
+  }
+
+  /**
+   * Returns a unique name for a file with a given content hash.
+   *
+   * <p>Directory paths are removed. Example:
+   * <pre>
+   * dir="a/b/c/d", contentHash="f000" => d-f000.jar
+   * file="a/b/c/d.txt", contentHash="f000" => d-f000.txt
+   * file="a/b/c/d", contentHash="f000" => d-f000
+   * </pre>
+   */
+  static String getUniqueContentName(File classpathElement, String contentHash) {
+    String fileName = Files.getNameWithoutExtension(classpathElement.getAbsolutePath());
+    String fileExtension = Files.getFileExtension(classpathElement.getAbsolutePath());
+    if (classpathElement.isDirectory()) {
+      return fileName + "-" + contentHash + ".jar";
+    } else if (fileExtension.isEmpty()) {
+      return fileName + "-" + contentHash;
+    }
+    return fileName + "-" + contentHash + "." + fileExtension;
+  }
+
+  /**
+   * Copies the contents of the classpathElement to the output channel.
+   *
+   * <p>If the classpathElement is a directory, a Zip stream is constructed on the fly,
+   * otherwise the file contents are copied as-is.
+   *
+   * <p>The output channel is not closed.
+   */
+  private static void copyContent(String classpathElement, WritableByteChannel outputChannel)
+      throws IOException {
+    final File classpathElementFile = new File(classpathElement);
+    if (classpathElementFile.isDirectory()) {
+      ZipFiles.zipDirectory(classpathElementFile, Channels.newOutputStream(outputChannel));
+    } else {
+      Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel));
+    }
+  }
+  /**
+   * Holds the metadata necessary to stage a file or confirm that a staged file has not changed.
+   */
+  static class PackageAttributes {
+    private final boolean directory;
+    private final long size;
+    private final String hash;
+    private DataflowPackage dataflowPackage;
+
+    public PackageAttributes(long size, String hash, boolean directory,
+        DataflowPackage dataflowPackage) {
+      this.size = size;
+      this.hash = Objects.requireNonNull(hash, "hash");
+      this.directory = directory;
+      this.dataflowPackage = Objects.requireNonNull(dataflowPackage, "dataflowPackage");
+    }
+
+    /**
+     * @return the dataflowPackage
+     */
+    public DataflowPackage getDataflowPackage() {
+      return dataflowPackage;
+    }
+
+    /**
+     * @return the directory
+     */
+    public boolean isDirectory() {
+      return directory;
+    }
+
+    /**
+     * @return the size
+     */
+    public long getSize() {
+      return size;
+    }
+
+    /**
+     * @return the hash
+     */
+    public String getHash() {
+      return hash;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
new file mode 100644
index 0000000..f6c6a71
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
@@ -0,0 +1,30 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import com.google.api.services.dataflow.model.DataflowPackage;
+
+import java.util.List;
+
+/**
+ * Interface for staging files needed for running a Dataflow pipeline.
+ */
+public interface Stager {
+  /* Stage files and return a list of packages. */
+  public List<DataflowPackage> stageFiles();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/io/DataflowTextIOTest.java
deleted file mode 100644
index 1bd8a85..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/io/DataflowTextIOTest.java
+++ /dev/null
@@ -1,118 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.common.collect.ImmutableList;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.List;
-
-/**
- * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write transforms.
- */
-@RunWith(JUnit4.class)
-public class DataflowTextIOTest {
-
-  private TestDataflowPipelineOptions buildTestPipelineOptions() {
-    TestDataflowPipelineOptions options =
-        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
-    options.setGcpCredential(new TestCredential());
-    return options;
-  }
-
-  private GcsUtil buildMockGcsUtil() throws IOException {
-    GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
-
-    // Any request to open gets a new bogus channel
-    Mockito
-        .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
-        .then(new Answer<SeekableByteChannel>() {
-          @Override
-          public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
-            return FileChannel.open(
-                Files.createTempFile("channel-", ".tmp"),
-                StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
-          }
-        });
-
-    // Any request for expansion returns a list containing the original GcsPath
-    // This is required to pass validation that occurs in TextIO during apply()
-    Mockito
-        .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
-        .then(new Answer<List<GcsPath>>() {
-          @Override
-          public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-            return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-          }
-        });
-
-    return mockGcsUtil;
-  }
-
-  /**
-   * This tests a few corner cases that should not crash.
-   */
-  @Test
-  public void testGoodWildcards() throws Exception {
-    TestDataflowPipelineOptions options = buildTestPipelineOptions();
-    options.setGcsUtil(buildMockGcsUtil());
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    applyRead(pipeline, "gs://bucket/foo");
-    applyRead(pipeline, "gs://bucket/foo/");
-    applyRead(pipeline, "gs://bucket/foo/*");
-    applyRead(pipeline, "gs://bucket/foo/?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]");
-    applyRead(pipeline, "gs://bucket/foo/*baz*");
-    applyRead(pipeline, "gs://bucket/foo/*baz?");
-    applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
-    applyRead(pipeline, "gs://bucket/foo/baz/*");
-    applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
-    applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
-    applyRead(pipeline, "gs://bucket/foo*/baz");
-    applyRead(pipeline, "gs://bucket/foo?/baz");
-    applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
-
-    // Check that running doesn't fail.
-    pipeline.run();
-  }
-
-  private void applyRead(Pipeline pipeline, String path) {
-    pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptionsTest.java
deleted file mode 100644
index 1b5a3c7..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptionsTest.java
+++ /dev/null
@@ -1,41 +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 com.google.cloud.dataflow.sdk.options;
-
-import static org.hamcrest.Matchers.hasEntry;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowPipelineDebugOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineDebugOptionsTest {
-  @Test
-  public void testTransformNameMapping() throws Exception {
-    DataflowPipelineDebugOptions options = PipelineOptionsFactory
-        .fromArgs(new String[]{"--transformNameMapping={\"a\":\"b\",\"foo\":\"\",\"bar\":\"baz\"}"})
-        .as(DataflowPipelineDebugOptions.class);
-    assertEquals(3, options.getTransformNameMapping().size());
-    assertThat(options.getTransformNameMapping(), hasEntry("a", "b"));
-    assertThat(options.getTransformNameMapping(), hasEntry("foo", ""));
-    assertThat(options.getTransformNameMapping(), hasEntry("bar", "baz"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java
deleted file mode 100644
index eff79bb..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java
+++ /dev/null
@@ -1,92 +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 com.google.cloud.dataflow.sdk.options;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import com.google.cloud.dataflow.sdk.testing.ResetDateTimeProvider;
-import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowPipelineOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineOptionsTest {
-  @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
-  @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider();
-
-  @Test
-  public void testJobNameIsSet() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    assertEquals("TestJobName", options.getJobName());
-  }
-
-  @Test
-  public void testUserNameIsNotSet() {
-    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
-    System.getProperties().remove("user.name");
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setAppName("TestApplication");
-    assertEquals("testapplication--1208190706", options.getJobName());
-    assertTrue(options.getJobName().length() <= 40);
-  }
-
-  @Test
-  public void testAppNameAndUserNameAreLong() {
-    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
-    System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setAppName("1234567890123456789012345678901234567890");
-    assertEquals(
-        "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706",
-        options.getJobName());
-  }
-
-  @Test
-  public void testAppNameIsLong() {
-    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
-    System.getProperties().put("user.name", "abcde");
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setAppName("1234567890123456789012345678901234567890");
-    assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName());
-  }
-
-  @Test
-  public void testUserNameIsLong() {
-    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
-    System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setAppName("1234567890");
-    assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName());
-  }
-
-  @Test
-  public void testUtf8UserNameAndApplicationNameIsNormalized() {
-    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
-    System.getProperties().put("user.name", "ði ıntəˈnæʃənəl ");
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn");
-    assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptionsTest.java
deleted file mode 100644
index 1420273..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptionsTest.java
+++ /dev/null
@@ -1,49 +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 com.google.cloud.dataflow.sdk.options;
-
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.hamcrest.Matchers;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link DataflowProfilingOptions}.
- */
-@RunWith(JUnit4.class)
-public class DataflowProfilingOptionsTest {
-
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-
-  @Test
-  public void testOptionsObject() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {
-        "--enableProfilingAgent", "--profilingAgentConfiguration={\"interval\": 21}"})
-        .as(DataflowPipelineOptions.class);
-    assertTrue(options.getEnableProfilingAgent());
-
-    String json = MAPPER.writeValueAsString(options);
-    assertThat(json, Matchers.containsString(
-        "\"profilingAgentConfiguration\":{\"interval\":21}"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java
deleted file mode 100644
index b752f3d..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java
+++ /dev/null
@@ -1,75 +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 com.google.cloud.dataflow.sdk.options;
-
-import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.WARN;
-import static org.junit.Assert.assertEquals;
-
-import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
-import com.google.common.collect.ImmutableMap;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-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 DataflowWorkerLoggingOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowWorkerLoggingOptionsTest {
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-
-  @Test
-  public void testWorkerLogLevelOverrideWithInvalidLogLevel() {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("Unsupported log level");
-    WorkerLogLevelOverrides.from(ImmutableMap.of("Name", "FakeLevel"));
-  }
-
-  @Test
-  public void testWorkerLogLevelOverrideForClass() throws Exception {
-    assertEquals("{\"org.junit.Test\":\"WARN\"}",
-        MAPPER.writeValueAsString(
-            new WorkerLogLevelOverrides().addOverrideForClass(Test.class, WARN)));
-  }
-
-  @Test
-  public void testWorkerLogLevelOverrideForPackage() throws Exception {
-    assertEquals("{\"org.junit\":\"WARN\"}",
-        MAPPER.writeValueAsString(
-            new WorkerLogLevelOverrides().addOverrideForPackage(Test.class.getPackage(), WARN)));
-  }
-
-  @Test
-  public void testWorkerLogLevelOverrideForName() throws Exception {
-    assertEquals("{\"A\":\"WARN\"}",
-        MAPPER.writeValueAsString(
-            new WorkerLogLevelOverrides().addOverrideForName("A", WARN)));
-  }
-
-  @Test
-  public void testSerializationAndDeserializationOf() throws Exception {
-    String testValue = "{\"A\":\"WARN\"}";
-    assertEquals(testValue,
-        MAPPER.writeValueAsString(
-            MAPPER.readValue(testValue, WorkerLogLevelOverrides.class)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java
deleted file mode 100644
index 0322426..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,302 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-
-import org.hamcrest.Description;
-import org.hamcrest.Factory;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for BlockingDataflowPipelineRunner.
- */
-@RunWith(JUnit4.class)
-public class BlockingDataflowPipelineRunnerTest {
-
-  @Rule
-  public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class);
-
-  @Rule
-  public ExpectedException expectedThrown = ExpectedException.none();
-
-  /**
-   * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
-   * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
-   */
-  private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
-      extends TypeSafeMatcher<T> {
-
-    private final Matcher<DataflowPipelineJob> matcher;
-
-    public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T ex) {
-      return matcher.matches(ex.getJob());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("job ");
-        matcher.describeMismatch(item.getMessage(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("exception with job matching ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowJobException> Matcher<T> expectJob(
-        Matcher<DataflowPipelineJob> matcher) {
-      return new DataflowJobExceptionMatcher<T>(matcher);
-    }
-  }
-
-  /**
-   * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
-   * to the return value of {@link DataflowPipelineJob#getJobId()}.
-   */
-  private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
-
-    private final Matcher<String> matcher;
-
-    public JobIdMatcher(Matcher<String> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T job) {
-      return matcher.matches(job.getJobId());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("jobId ");
-        matcher.describeMismatch(item.getJobId(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("job with jobId ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
-      return new JobIdMatcher<T>(equalTo(jobId));
-    }
-  }
-
-  /**
-   * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
-   * {@link Matcher} to the {@link DataflowPipelineJob} returned by
-   * {@link DataflowJobUpdatedException#getReplacedByJob()}.
-   */
-  private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
-      extends TypeSafeMatcher<T> {
-
-    private final Matcher<DataflowPipelineJob> matcher;
-
-    public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T ex) {
-      return matcher.matches(ex.getReplacedByJob());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("job ");
-        matcher.describeMismatch(item.getMessage(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("exception with replacedByJob() ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
-        Matcher<DataflowPipelineJob> matcher) {
-      return new ReplacedByJobMatcher<T>(matcher);
-    }
-  }
-
-  /**
-   * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
-   * that will immediately terminate in the provided {@code terminalState}.
-   *
-   * <p>The return value may be further mocked.
-   */
-  private DataflowPipelineJob createMockJob(
-      String projectId, String jobId, State terminalState) throws Exception {
-    DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
-    when(mockJob.getProjectId()).thenReturn(projectId);
-    when(mockJob.getJobId()).thenReturn(jobId);
-    when(mockJob.waitToFinish(
-        anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
-        .thenReturn(terminalState);
-    return mockJob;
-  }
-
-  /**
-   * Returns a {@link BlockingDataflowPipelineRunner} that will return the provided a job to return.
-   * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
-   */
-  private BlockingDataflowPipelineRunner createMockRunner(DataflowPipelineJob job)
-      throws Exception {
-    DataflowPipelineRunner mockRunner = mock(DataflowPipelineRunner.class);
-    TestDataflowPipelineOptions options =
-        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
-    options.setProject(job.getProjectId());
-
-    when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
-
-    return new BlockingDataflowPipelineRunner(mockRunner, options);
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} returns normally when a job terminates in
-   * the {@link State#DONE DONE} state.
-   */
-  @Test
-  public void testJobDoneComplete() throws Exception {
-    createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
-        .run(DirectPipeline.createForTest());
-    expectedLogs.verifyInfo("Job finished with status DONE");
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#FAILED FAILED} state.
-   */
-  @Test
-  public void testFailedJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobExecutionException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testFailedJob-jobId")));
-    createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
-        .run(DirectPipeline.createForTest());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
-   */
-  @Test
-  public void testCancelledJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobCancelledException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testCancelledJob-jobId")));
-    createMockRunner(
-        createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
-        .run(DirectPipeline.createForTest());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#UPDATED UPDATED} state.
-   */
-  @Test
-  public void testUpdatedJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobUpdatedException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
-    expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
-        JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
-    DataflowPipelineJob job =
-        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
-    DataflowPipelineJob replacedByJob =
-        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
-    when(job.getReplacedByJob()).thenReturn(replacedByJob);
-    createMockRunner(job).run(DirectPipeline.createForTest());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
-   * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
-   * is an old SDK relative the service).
-   */
-  @Test
-  public void testUnknownJobThrowsException() throws Exception {
-    expectedThrown.expect(IllegalStateException.class);
-    createMockRunner(
-        createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
-        .run(DirectPipeline.createForTest());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job returns a {@code null} state, indicating that it failed to contact the service,
-   * including all of its built-in resilience logic.
-   */
-  @Test
-  public void testNullJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowServiceException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testNullJob-jobId")));
-    createMockRunner(
-        createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
-        .run(DirectPipeline.createForTest());
-  }
-
-  @Test
-  public void testToString() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    assertEquals("BlockingDataflowPipelineRunner#TestJobName",
-        BlockingDataflowPipelineRunner.fromOptions(options).toString());
-  }
-}


[61/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
index f19af3b..132d7f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.BinaryCombineFn;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind;
-import com.google.cloud.dataflow.sdk.util.common.CounterProvider;
+import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterProvider;
 
 import java.io.Serializable;
 import java.util.Comparator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java
index 74b4a8c..5005f6b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.DoubleCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
+
 import com.google.common.base.MoreObjects;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
index 26212f6..dcee91f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.BinaryCombineFn;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind;
-import com.google.cloud.dataflow.sdk.util.common.CounterProvider;
+import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterProvider;
 
 import java.io.Serializable;
 import java.util.Comparator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 8ef28da..d502b93 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
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
-import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.sdk.util.StringUtils;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.TypedPValue;
 
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
@@ -37,14 +37,14 @@ import java.io.Serializable;
  * {@code OutputT} (some subtype of {@link POutput}).
  *
  * <p>Common PTransforms include root PTransforms like
- * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read},
+ * {@link org.apache.beam.sdk.io.TextIO.Read},
  * {@link Create}, processing and
  * conversion operations like {@link ParDo},
  * {@link GroupByKey},
- * {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey},
+ * {@link org.apache.beam.sdk.transforms.join.CoGroupByKey},
  * {@link Combine}, and {@link Count}, and outputting
  * PTransforms like
- * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}.  Users also
+ * {@link org.apache.beam.sdk.io.TextIO.Write}.  Users also
  * define their own application-specific composite PTransforms.
  *
  * <p>Each {@code PTransform<InputT, OutputT>} has a single
@@ -52,18 +52,18 @@ import java.io.Serializable;
  * PTransforms conceptually transform one input value to one output
  * value, and in this case {@code InputT} and {@code Output} are
  * typically instances of
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * {@link org.apache.beam.sdk.values.PCollection}.
  * A root
  * PTransform conceptually has no input; in this case, conventionally
- * a {@link com.google.cloud.dataflow.sdk.values.PBegin} object
+ * a {@link org.apache.beam.sdk.values.PBegin} object
  * produced by calling {@link Pipeline#begin} is used as the input.
  * An outputting PTransform conceptually has no output; in this case,
- * conventionally {@link com.google.cloud.dataflow.sdk.values.PDone}
+ * conventionally {@link org.apache.beam.sdk.values.PDone}
  * is used as its output type.  Some PTransforms conceptually have
  * multiple inputs and/or outputs; in these cases special "bundling"
  * classes like
- * {@link com.google.cloud.dataflow.sdk.values.PCollectionList},
- * {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple}
+ * {@link org.apache.beam.sdk.values.PCollectionList},
+ * {@link org.apache.beam.sdk.values.PCollectionTuple}
  * are used
  * to combine multiple values into a single bundle for passing into or
  * returning from the PTransform.
@@ -100,7 +100,7 @@ import java.io.Serializable;
  * its own name derived from the name of its producing PTransform.
  *
  * <p>Each PCollection output produced by a PTransform
- * also records a {@link com.google.cloud.dataflow.sdk.coders.Coder}
+ * also records a {@link org.apache.beam.sdk.coders.Coder}
  * that specifies how the elements of that PCollection
  * are to be encoded as a byte string, if necessary.  The
  * PTransform may provide a default Coder for any of its outputs, for
@@ -109,17 +109,17 @@ import java.io.Serializable;
  * the system will attempt to infer a Coder for it, based on
  * what's known at run-time about the Java type of the output's
  * elements.  The enclosing {@link Pipeline}'s
- * {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry}
+ * {@link org.apache.beam.sdk.coders.CoderRegistry}
  * (accessible via {@link Pipeline#getCoderRegistry}) defines the
  * mapping from Java types to the default Coder to use, for a standard
  * set of Java types; users can extend this mapping for additional
  * types, via
- * {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry#registerCoder}.
+ * {@link org.apache.beam.sdk.coders.CoderRegistry#registerCoder}.
  * If this inference process fails, either because the Java type was
  * not known at run-time (e.g., due to Java's "erasure" of generic
  * types) or there was no default Coder registered, then the Coder
  * should be specified manually by calling
- * {@link com.google.cloud.dataflow.sdk.values.TypedPValue#setCoder}
+ * {@link org.apache.beam.sdk.values.TypedPValue#setCoder}
  * on the output PCollection.  The Coder of every output
  * PCollection must be determined one way or another
  * before that output is used as an input to another PTransform, or
@@ -185,7 +185,7 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
    * <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 com.google.cloud.dataflow.sdk.runners.PipelineRunner#apply}.
+   * {@link org.apache.beam.sdk.runners.PipelineRunner#apply}.
    */
   public OutputT apply(InputT input) {
     throw new IllegalArgumentException(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index 8f5d26c..b448c26 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -15,38 +15,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
+import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.DirectModeExecutionContext;
+import org.apache.beam.sdk.util.DirectSideInputReader;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunnerBase;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.PTuple;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.StringUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.apache.beam.sdk.values.TypedPValue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext;
-import com.google.cloud.dataflow.sdk.util.DirectSideInputReader;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunnerBase;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.MutationDetector;
-import com.google.cloud.dataflow.sdk.util.MutationDetectors;
-import com.google.cloud.dataflow.sdk.util.PTuple;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java
index 60e1d49..47d49f7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import java.io.Serializable;
 
@@ -56,7 +56,7 @@ import java.io.Serializable;
  * <p>Each output element has the same timestamp and is in the same windows
  * as its corresponding input element, and each output {@code PCollection}
  * has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * associated with it as the input.
  *
  * @param <T> the type of the elements of the input and output

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java
index c9738a3..84b1f80 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
  * {@code RemoveDuplicates<T>} takes a {@code PCollection<T>} and
@@ -43,7 +43,7 @@ import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
  * <p>Each output element is in the same window as its corresponding input
  * element, and has the timestamp of the end of that window.  The output
  * {@code PCollection} has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * as the input.
  *
  * <p>Does not preserve any order the input PCollection might have had.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 83c7f2e..1e621d4 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
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java
index 52cdaa0..390b852 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableComparator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import java.io.Serializable;
 import java.util.Comparator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java
index d1da82d..eb15630 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
index bb8ea4c..a879925 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
  * A {@link SerializableFunction} which is not a <i>functional interface</i>.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java
index 7144aa3..07f78b5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind;
-import com.google.cloud.dataflow.sdk.util.common.CounterProvider;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterProvider;
 
 /**
  * {@code PTransform}s for computing the sum of the elements in a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b81c152..82747c2 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
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.PerKey;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn;
+import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
+import org.apache.beam.sdk.transforms.Combine.PerKey;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java
index 3fc69ca..0b533b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * {@code Values<V>} takes a {@code PCollection} of {@code KV<K, V>}s and
@@ -33,7 +33,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection;
  * <p>Each output element has the same timestamp and is in the same windows
  * as its corresponding input element, and the output {@code PCollection}
  * has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * associated with it as the input.
  *
  * <p>See also {@link Keys}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
index 6f40889..3df915b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.util.PCollectionViews;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import java.util.List;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
index 91ef254..198e7cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
  * {@code WithKeys<K, V>} takes a {@code PCollection<V>}, and either a
@@ -43,7 +43,7 @@ import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
  * <p>Each output element has the same timestamp and is in the same windows
  * as its corresponding input element, and the output {@code PCollection}
  * has the same
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}
  * associated with it as the input.
  *
  * @param <K> the type of the keys in the output {@code PCollection}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java
index bc8e047..eae8de5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -32,10 +32,10 @@ import org.joda.time.Instant;
  * A {@link PTransform} for assigning timestamps to all the elements of a {@link PCollection}.
  *
  * <p>Timestamps are used to assign {@link BoundedWindow Windows} to elements within the
- * {@link Window#into(com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn)}
+ * {@link Window#into(org.apache.beam.sdk.transforms.windowing.WindowFn)}
  * {@link PTransform}. Assigning timestamps is useful when the input data set comes from a
  * {@link Source} without implicit timestamps (such as
- * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read TextIO}).
+ * {@link org.apache.beam.sdk.io.TextIO.Read TextIO}).
  *
  */
 public class WithTimestamps<T> extends PTransform<PCollection<T>, PCollection<T>> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Write.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Write.java
index 7c6f677..417da44 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Write.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Write.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 /**
  * A backwards-compatible {@code Write} class that simply inherits from the
- * {@link com.google.cloud.dataflow.sdk.io.Write} class that should be used instead.
+ * {@link org.apache.beam.sdk.io.Write} class that should be used instead.
  *
- * @deprecated: use {@link com.google.cloud.dataflow.sdk.io.Write} from the
- * {@code com.google.cloud.dataflow.sdk.io} package instead.
+ * @deprecated: use {@link org.apache.beam.sdk.io.Write} from the
+ * {@code org.apache.beam.sdk.io} package instead.
  */
 @Deprecated
-public class Write extends com.google.cloud.dataflow.sdk.io.Write {
+public class Write extends org.apache.beam.sdk.io.Write {
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 d9098ba..3aeed83 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.display;
+package org.apache.beam.sdk.transforms.display;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java
index 36f7a31..b295d97 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.display;
+package org.apache.beam.sdk.transforms.display;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * Marker interface for {@link PTransform PTransforms} and components used within

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c7aef32..3732bec 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
@@ -15,19 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.Reiterator;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+package org.apache.beam.sdk.transforms.join;
+
+import static org.apache.beam.sdk.util.Structs.addObject;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.common.Reiterator;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java
index bed8a5b..29a0c7d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addList;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java
index 6ac76d6..d7ac5e4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult.CoGbkResultCoder;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple.TaggedKeyedPCollection;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+package org.apache.beam.sdk.transforms.join;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+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.join.CoGbkResult.CoGbkResultCoder;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple.TaggedKeyedPCollection;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -38,7 +38,7 @@ import java.util.List;
  * of tables.  A {@link CoGroupByKey} groups results from all
  * tables by like keys into {@link CoGbkResult}s,
  * from which the results for any specific table can be accessed by the
- * {@link com.google.cloud.dataflow.sdk.values.TupleTag}
+ * {@link org.apache.beam.sdk.values.TupleTag}
  * supplied with the initial table.
  *
  * <p>Example of performing a {@link CoGroupByKey} followed by a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
index 1a5758c..0852c9c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+package org.apache.beam.sdk.transforms.join;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/RawUnionValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/RawUnionValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/RawUnionValue.java
index a74ee52..159b100 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/RawUnionValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/RawUnionValue.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
 // TODO: Think about making this a complete dynamic union by adding
 // a schema.  Type would then be defined by the corresponding schema entry.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
index 8e6206c..43f1adf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+package org.apache.beam.sdk.transforms.join;
+
+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.PropertyNames;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/package-info.java
index 7672a3a..f4b315e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/package-info.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Defines the {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey} transform
+ * Defines the {@link org.apache.beam.sdk.transforms.join.CoGroupByKey} transform
  * for joining multiple PCollections.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/package-info.java
index 68d602a..892dee9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/package-info.java
@@ -16,28 +16,28 @@
  * limitations under the License.
  */
 /**
- * Defines {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s for transforming
+ * Defines {@link org.apache.beam.sdk.transforms.PTransform}s for transforming
  * data in a pipeline.
  *
- * <p>A {@link com.google.cloud.dataflow.sdk.transforms.PTransform} is an operation that takes an
- * {@code InputT} (some subtype of {@link com.google.cloud.dataflow.sdk.values.PInput})
+ * <p>A {@link org.apache.beam.sdk.transforms.PTransform} is an operation that takes an
+ * {@code InputT} (some subtype of {@link org.apache.beam.sdk.values.PInput})
  * and produces an
- * {@code OutputT} (some subtype of {@link com.google.cloud.dataflow.sdk.values.POutput}).
+ * {@code OutputT} (some subtype of {@link org.apache.beam.sdk.values.POutput}).
  *
  * <p>Common PTransforms include root PTransforms like
- * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read} and
- * {@link com.google.cloud.dataflow.sdk.transforms.Create}, processing and
- * conversion operations like {@link com.google.cloud.dataflow.sdk.transforms.ParDo},
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey},
- * {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey},
- * {@link com.google.cloud.dataflow.sdk.transforms.Combine}, and
- * {@link com.google.cloud.dataflow.sdk.transforms.Count}, and outputting
+ * {@link org.apache.beam.sdk.io.TextIO.Read} and
+ * {@link org.apache.beam.sdk.transforms.Create}, processing and
+ * conversion operations like {@link org.apache.beam.sdk.transforms.ParDo},
+ * {@link org.apache.beam.sdk.transforms.GroupByKey},
+ * {@link org.apache.beam.sdk.transforms.join.CoGroupByKey},
+ * {@link org.apache.beam.sdk.transforms.Combine}, and
+ * {@link org.apache.beam.sdk.transforms.Count}, and outputting
  * PTransforms like
- * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}.
+ * {@link org.apache.beam.sdk.io.TextIO.Write}.
  *
  * <p>New PTransforms can be created by composing existing PTransforms.
  * Most PTransforms in this package are composites, and users can also create composite PTransforms
  * for their own application-specific logic.
  *
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 aa93ec6..ac1fa43 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+
 import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
index 34f7056..83e0bea 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
@@ -15,22 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.state.AccumulatorCombiningState;
-import com.google.cloud.dataflow.sdk.util.state.CombiningState;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateMerging;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
index 46b10da..0c80851 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0a4f885..1462ec4 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+
 import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
index 5a261ee..977259d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.state.AccumulatorCombiningState;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateMerging;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
index 97cefa3..b2ea1b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.TimeDomain;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
index 4ccba04..e8e0fb6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
+
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.TimeDomain;
 
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
 import com.google.common.base.Objects;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 e62bce2..5aca093 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
@@ -37,14 +38,14 @@ import java.util.Objects;
  * by the pipeline.
  *
  * <p>For sources that provide non-heuristic watermarks (e.g.
- * {@link com.google.cloud.dataflow.sdk.io.PubsubIO} when using arrival times as event times), the
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
  * watermark is a strict guarantee that no data with an event time earlier than
  * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
  * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
  * of the window will be the last pane ever for that window.
  *
  * <p>For sources that provide heuristic watermarks (e.g.
- * {@link com.google.cloud.dataflow.sdk.io.PubsubIO} when using user-supplied event times), the
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
  * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
  * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
  * often be quite accurate, but the chance of seeing late data for any given window is non-zero.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
index 7b66f7e..50ee9b5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java
index 2da0c03..794d5fd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
index 90c7211..3c6dbf3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimeDomain;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java
index 5671fd8..cc43887 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java
index c4c5183..a4712fe 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.AtomicCoder;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
index 734c212..499ffeb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java
index 88d11e0..6f9c717 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.DurationCoder;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.DurationCoder;
+import org.apache.beam.sdk.coders.InstantCoder;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
index 23f4a3f..e0d65e5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
index 0d12c75..8314dd7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import java.util.ArrayList;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/NonMergingWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/NonMergingWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/NonMergingWindowFn.java
index 03256d0..d304a68 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/NonMergingWindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/NonMergingWindowFn.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 /**
  * Abstract base class for {@link WindowFn}s that do not merge windows.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
index 68b9083..48e1dc2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
+
+import org.apache.beam.sdk.util.ExecutableTrigger;
 
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java
index 76c8a11..3deea56 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
+
 import com.google.common.collect.Ordering;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
index a080991..363181b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
 import org.apache.beam.sdk.annotations.Experimental;
+
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 656b87f..266c4d3 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
-
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.util.VarInt;
+package org.apache.beam.sdk.transforms.windowing;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.DoFn;
+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;
@@ -35,7 +36,7 @@ import java.util.Objects;
 /**
  * Provides information about the pane an element belongs to. Every pane is implicitly associated
  * with a window. Panes are observable only via the
- * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.ProcessContext#pane} method of the context
+ * {@link org.apache.beam.sdk.transforms.DoFn.ProcessContext#pane} method of the context
  * passed to a {@link DoFn#processElement} overridden method.
  *
  * <p>Note: This does not uniquely identify a pane, and should not be used for comparisons.
@@ -64,16 +65,16 @@ public final class PaneInfo {
    * produces a final pane, it will not be merged into any new windows.
    *
    * <p>The predictions above are made using the mechanism of watermarks.
-   * See {@link com.google.cloud.dataflow.sdk.util.TimerInternals} for more information
+   * See {@link org.apache.beam.sdk.util.TimerInternals} for more information
    * about watermarks.
    *
    * <p>We can state some properties of {@code LATE} and {@code ON_TIME} panes, but first need some
    * definitions:
    * <ol>
    * <li>We'll call a pipeline 'simple' if it does not use
-   * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp} in
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#outputWithTimestamp} in
    * any {@code DoFn}, and it uses the same
-   * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound#withAllowedLateness}
+   * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound#withAllowedLateness}
    * argument value on all windows (or uses the default of {@link org.joda.time.Duration#ZERO}).
    * <li>We'll call an element 'locally late', from the point of view of a computation on a
    * worker, if the element's timestamp is before the input watermark for that computation

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
index a756ed9..b0dd8b9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
index 988f0a4..414f107 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
index bf2a487..3be6454 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Duration;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
index 7c2a39e..3a7b072 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;


[32/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
new file mode 100644
index 0000000..2818251
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
@@ -0,0 +1,3009 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static com.google.cloud.dataflow.sdk.util.StringUtils.approximatePTransformName;
+import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
+import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInEmptyWindows;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.services.clouddebugger.v2.Clouddebugger;
+import com.google.api.services.clouddebugger.v2.model.Debuggee;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.api.services.dataflow.model.WorkerPool;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
+import com.google.cloud.dataflow.sdk.coders.CoderException;
+import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
+import com.google.cloud.dataflow.sdk.coders.IterableCoder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.ListCoder;
+import com.google.cloud.dataflow.sdk.coders.MapCoder;
+import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
+import com.google.cloud.dataflow.sdk.coders.StandardCoder;
+import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
+import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.FileBasedSink;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.io.Write;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
+import com.google.cloud.dataflow.sdk.options.StreamingOptions;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.JobSpecification;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import com.google.cloud.dataflow.sdk.runners.dataflow.AssignWindows;
+import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
+import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
+import com.google.cloud.dataflow.sdk.transforms.WithKeys;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
+import com.google.cloud.dataflow.sdk.util.DataflowTransport;
+import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
+import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.util.PCollectionViews;
+import com.google.cloud.dataflow.sdk.util.PathValidator;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.Reshuffle;
+import com.google.cloud.dataflow.sdk.util.SystemDoFnInternal;
+import com.google.cloud.dataflow.sdk.util.ValueWithRecordId;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.cloud.dataflow.sdk.values.TupleTagList;
+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;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.format.DateTimeFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to the Dataflow representation
+ * using the {@link DataflowPipelineTranslator} and then submitting
+ * them to a Dataflow service for execution.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code DataflowPipelineRunner}, the Google cloudservices account and the Google compute engine
+ * service account of the GCP project running the Dataflow Job will need access to the corresponding
+ * source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class);
+
+  /** Provided configuration options. */
+  private final DataflowPipelineOptions options;
+
+  /** Client for the Dataflow service. This is used to actually submit jobs. */
+  private final Dataflow dataflowClient;
+
+  /** Translator for this DataflowPipelineRunner, based on options. */
+  private final DataflowPipelineTranslator translator;
+
+  /** Custom transforms implementations. */
+  private final Map<Class<?>, Class<?>> overrides;
+
+  /** A set of user defined functions to invoke at different points in execution. */
+  private DataflowPipelineRunnerHooks hooks;
+
+  // Environment version information.
+  private static final String ENVIRONMENT_MAJOR_VERSION = "4";
+
+  // Default Docker container images that execute Dataflow worker harness, residing in Google
+  // Container Registry, separately for Batch and Streaming.
+  public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
+      = "dataflow.gcr.io/v1beta3/java-batch:1.5.1";
+  public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
+      = "dataflow.gcr.io/v1beta3/java-streaming:1.5.1";
+
+  // The limit of CreateJob request size.
+  private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
+
+  private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
+
+  /**
+   * Project IDs must contain lowercase letters, digits, or dashes.
+   * IDs must start with a letter and may not end with a dash.
+   * This regex isn't exact - this allows for patterns that would be rejected by
+   * the service, but this is sufficient for basic validation of project IDs.
+   */
+  public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
+
+  /**
+   * Construct a runner from the provided options.
+   *
+   * @param options Properties that configure the runner.
+   * @return The newly created runner.
+   */
+  public static DataflowPipelineRunner fromOptions(PipelineOptions options) {
+    // (Re-)register standard IO factories. Clobbers any prior credentials.
+    IOChannelUtils.registerStandardIOFactories(options);
+
+    DataflowPipelineOptions dataflowOptions =
+        PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
+    ArrayList<String> missing = new ArrayList<>();
+
+    if (dataflowOptions.getAppName() == null) {
+      missing.add("appName");
+    }
+    if (missing.size() > 0) {
+      throw new IllegalArgumentException(
+          "Missing required values: " + Joiner.on(',').join(missing));
+    }
+
+    PathValidator validator = dataflowOptions.getPathValidator();
+    Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
+        && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
+        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+    if (dataflowOptions.getStagingLocation() != null) {
+      validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
+    }
+    if (dataflowOptions.getTempLocation() != null) {
+      validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
+    }
+    if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
+      dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
+    } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
+      try {
+        dataflowOptions.setStagingLocation(
+            IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
+            + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
+      }
+    }
+
+    if (dataflowOptions.getFilesToStage() == null) {
+      dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
+          DataflowPipelineRunner.class.getClassLoader()));
+      LOG.info("PipelineOptions.filesToStage was not specified. "
+          + "Defaulting to files from the classpath: will stage {} files. "
+          + "Enable logging at DEBUG level to see which files will be staged.",
+          dataflowOptions.getFilesToStage().size());
+      LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
+    }
+
+    // Verify jobName according to service requirements.
+    String jobName = dataflowOptions.getJobName().toLowerCase();
+    Preconditions.checkArgument(
+        jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
+        "JobName invalid; the name must consist of only the characters "
+            + "[-a-z0-9], starting with a letter and ending with a letter "
+            + "or number");
+
+    // Verify project
+    String project = dataflowOptions.getProject();
+    if (project.matches("[0-9]*")) {
+      throw new IllegalArgumentException("Project ID '" + project
+          + "' invalid. Please make sure you specified the Project ID, not project number.");
+    } else if (!project.matches(PROJECT_ID_REGEXP)) {
+      throw new IllegalArgumentException("Project ID '" + project
+          + "' invalid. Please make sure you specified the Project ID, not project description.");
+    }
+
+    DataflowPipelineDebugOptions debugOptions =
+        dataflowOptions.as(DataflowPipelineDebugOptions.class);
+    // Verify the number of worker threads is a valid value
+    if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
+      throw new IllegalArgumentException("Number of worker harness threads '"
+          + debugOptions.getNumberOfWorkerHarnessThreads()
+          + "' invalid. Please make sure the value is non-negative.");
+    }
+
+    return new DataflowPipelineRunner(dataflowOptions);
+  }
+
+  @VisibleForTesting protected DataflowPipelineRunner(DataflowPipelineOptions options) {
+    this.options = options;
+    this.dataflowClient = options.getDataflowClient();
+    this.translator = DataflowPipelineTranslator.fromOptions(options);
+    this.pcollectionsRequiringIndexedFormat = new HashSet<>();
+    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
+
+    if (options.isStreaming()) {
+      overrides = ImmutableMap.<Class<?>, Class<?>>builder()
+          .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class)
+          .put(Create.Values.class, StreamingCreate.class)
+          .put(View.AsMap.class, StreamingViewAsMap.class)
+          .put(View.AsMultimap.class, StreamingViewAsMultimap.class)
+          .put(View.AsSingleton.class, StreamingViewAsSingleton.class)
+          .put(View.AsList.class, StreamingViewAsList.class)
+          .put(View.AsIterable.class, StreamingViewAsIterable.class)
+          .put(Write.Bound.class, StreamingWrite.class)
+          .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class)
+          .put(Read.Unbounded.class, StreamingUnboundedRead.class)
+          .put(Read.Bounded.class, UnsupportedIO.class)
+          .put(AvroIO.Read.Bound.class, UnsupportedIO.class)
+          .put(AvroIO.Write.Bound.class, UnsupportedIO.class)
+          .put(BigQueryIO.Read.Bound.class, UnsupportedIO.class)
+          .put(TextIO.Read.Bound.class, UnsupportedIO.class)
+          .put(TextIO.Write.Bound.class, UnsupportedIO.class)
+          .put(Window.Bound.class, AssignWindows.class)
+          .build();
+    } else {
+      ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
+      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);
+      if (options.getExperiments() == null
+          || !options.getExperiments().contains("disable_ism_side_input")) {
+        builder.put(View.AsMap.class, BatchViewAsMap.class);
+        builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
+        builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
+        builder.put(View.AsList.class, BatchViewAsList.class);
+        builder.put(View.AsIterable.class, BatchViewAsIterable.class);
+      }
+      overrides = builder.build();
+    }
+  }
+
+  /**
+   * Applies the given transform to the input. For transforms with customized definitions
+   * for the Dataflow pipeline runner, the application is intercepted and modified here.
+   */
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+
+    if (Combine.GroupedValues.class.equals(transform.getClass())
+        || GroupByKey.class.equals(transform.getClass())) {
+
+      // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
+      // primitives. Returning a primitive output instead of the expanded definition
+      // signals to the translator that translation is necessary.
+      @SuppressWarnings("unchecked")
+      PCollection<?> pc = (PCollection<?>) input;
+      @SuppressWarnings("unchecked")
+      OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
+          pc.getPipeline(),
+          transform instanceof GroupByKey
+              ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
+              : pc.getWindowingStrategy(),
+          pc.isBounded());
+      return outputT;
+    } else if (Window.Bound.class.equals(transform.getClass())) {
+      /*
+       * TODO: make this the generic way overrides are applied (using super.apply() rather than
+       * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
+       * additional nodes into the graph.
+       */
+      // casting to wildcard
+      @SuppressWarnings("unchecked")
+      OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
+      return windowed;
+    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
+        && ((PCollectionList<?>) input).size() == 0) {
+      return (OutputT) Pipeline.applyTransform(input, Create.of());
+    } else if (overrides.containsKey(transform.getClass())) {
+      // It is the responsibility of whoever constructs overrides to ensure this is type safe.
+      @SuppressWarnings("unchecked")
+      Class<PTransform<InputT, OutputT>> transformClass =
+          (Class<PTransform<InputT, OutputT>>) transform.getClass();
+
+      @SuppressWarnings("unchecked")
+      Class<PTransform<InputT, OutputT>> customTransformClass =
+          (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
+
+      PTransform<InputT, OutputT> customTransform =
+          InstanceBuilder.ofType(customTransformClass)
+          .withArg(DataflowPipelineRunner.class, this)
+          .withArg(transformClass, transform)
+          .build();
+
+      return Pipeline.applyTransform(input, customTransform);
+    } else {
+      return super.apply(transform, input);
+    }
+  }
+
+  private <T> PCollection<T> applyWindow(
+      Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
+    // types are matched at compile time
+    @SuppressWarnings("unchecked")
+    Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
+    @SuppressWarnings("unchecked")
+    PCollection<T> input = (PCollection<T>) initialInput;
+    return super.apply(new AssignWindows<>(transform), input);
+  }
+
+  private String debuggerMessage(String projectId, String uniquifier) {
+    return String.format("To debug your job, visit Google Cloud Debugger at: "
+        + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
+        projectId, uniquifier);
+  }
+
+  private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
+    if (!options.getEnableCloudDebugger()) {
+      return;
+    }
+
+    if (options.getDebuggee() != null) {
+      throw new RuntimeException("Should not specify the debuggee");
+    }
+
+    Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
+    Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
+    options.setDebuggee(debuggee);
+
+    System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
+  }
+
+  private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
+    RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
+    registerReq.setDebuggee(new Debuggee()
+        .setProject(options.getProject())
+        .setUniquifier(uniquifier)
+        .setDescription(uniquifier)
+        .setAgentVersion("google.com/cloud-dataflow-java/v1"));
+
+    try {
+      RegisterDebuggeeResponse registerResponse =
+          debuggerClient.controller().debuggees().register(registerReq).execute();
+      Debuggee debuggee = registerResponse.getDebuggee();
+      if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
+        throw new RuntimeException("Unable to register with the debugger: " +
+            debuggee.getStatus().getDescription().getFormat());
+      }
+
+      return debuggee;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to register with the debugger: ", e);
+    }
+  }
+
+  @Override
+  public DataflowPipelineJob run(Pipeline pipeline) {
+    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
+
+    LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
+        + "related to Google Compute Engine usage and other Google Cloud Services.");
+
+    List<DataflowPackage> packages = options.getStager().stageFiles();
+
+
+    // Set a unique client_request_id in the CreateJob request.
+    // This is used to ensure idempotence of job creation across retried
+    // attempts to create a job. Specifically, if the service returns a job with
+    // a different client_request_id, it means the returned one is a different
+    // job previously created with the same job name, and that the job creation
+    // has been effectively rejected. The SDK should return
+    // Error::Already_Exists to user in that case.
+    int randomNum = new Random().nextInt(9000) + 1000;
+    String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
+        .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
+
+    // Try to create a debuggee ID. This must happen before the job is translated since it may
+    // update the options.
+    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+    maybeRegisterDebuggee(dataflowOptions, requestId);
+
+    JobSpecification jobSpecification =
+        translator.translate(pipeline, this, packages);
+    Job newJob = jobSpecification.getJob();
+    newJob.setClientRequestId(requestId);
+
+    String version = DataflowReleaseInfo.getReleaseInfo().getVersion();
+    System.out.println("Dataflow SDK version: " + version);
+
+    newJob.getEnvironment().setUserAgent(DataflowReleaseInfo.getReleaseInfo());
+    // The Dataflow Service may write to the temporary directory directly, so
+    // must be verified.
+    if (!Strings.isNullOrEmpty(options.getTempLocation())) {
+      newJob.getEnvironment().setTempStoragePrefix(
+          dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
+    }
+    newJob.getEnvironment().setDataset(options.getTempDatasetId());
+    newJob.getEnvironment().setExperiments(options.getExperiments());
+
+    // Set the Docker container image that executes Dataflow worker harness, residing in Google
+    // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
+    String workerHarnessContainerImage =
+        options.as(DataflowPipelineWorkerPoolOptions.class)
+        .getWorkerHarnessContainerImage();
+    for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
+      workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
+    }
+
+    // Requirements about the service.
+    Map<String, Object> environmentVersion = new HashMap<>();
+    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
+    newJob.getEnvironment().setVersion(environmentVersion);
+    // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
+    // autoscale if specified.
+    String jobType = "JAVA_BATCH_AUTOSCALING";
+
+    if (options.isStreaming()) {
+      jobType = "STREAMING";
+    }
+    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
+
+    if (hooks != null) {
+      hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
+    }
+
+    if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
+      try (PrintWriter printWriter = new PrintWriter(
+          new File(options.getDataflowJobFile()))) {
+        String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
+        printWriter.print(workSpecJson);
+        LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
+      } catch (IllegalStateException ex) {
+        LOG.warn("Cannot translate workflow spec to json for debug.");
+      } catch (FileNotFoundException ex) {
+        LOG.warn("Cannot create workflow spec output file.");
+      }
+    }
+
+    String jobIdToUpdate = null;
+    if (options.getUpdate()) {
+      jobIdToUpdate = getJobIdFromName(options.getJobName());
+      newJob.setTransformNameMapping(options.getTransformNameMapping());
+      newJob.setReplaceJobId(jobIdToUpdate);
+    }
+    Job jobResult;
+    try {
+      jobResult = dataflowClient
+              .projects()
+              .jobs()
+              .create(options.getProject(), newJob)
+              .execute();
+    } catch (GoogleJsonResponseException e) {
+      String errorMessages = "Unexpected errors";
+      if (e.getDetails() != null) {
+        if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
+          errorMessages = "The size of the serialized JSON representation of the pipeline "
+              + "exceeds the allowable limit. "
+              + "For more information, please check the FAQ link below:\n"
+              + "https://cloud.google.com/dataflow/faq";
+        } else {
+          errorMessages = e.getDetails().getMessage();
+        }
+      }
+      throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to create a workflow job", e);
+    }
+
+    // Obtain all of the extractors from the PTransforms used in the pipeline so the
+    // DataflowPipelineJob has access to them.
+    AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
+    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
+        aggregatorExtractor.getAggregatorSteps();
+
+    DataflowAggregatorTransforms aggregatorTransforms =
+        new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
+
+    // Use a raw client for post-launch monitoring, as status calls may fail
+    // regularly and need not be retried automatically.
+    DataflowPipelineJob dataflowPipelineJob =
+        new DataflowPipelineJob(options.getProject(), jobResult.getId(),
+            DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
+
+    // If the service returned client request id, the SDK needs to compare it
+    // with the original id generated in the request, if they are not the same
+    // (i.e., the returned job is not created by this request), throw
+    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
+    // depending on whether this is a reload or not.
+    if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
+        && !jobResult.getClientRequestId().equals(requestId)) {
+      // If updating a job.
+      if (options.getUpdate()) {
+        throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
+            String.format("The job named %s with id: %s has already been updated into job id: %s "
+                + "and cannot be updated again.",
+                newJob.getName(), jobIdToUpdate, jobResult.getId()));
+      } else {
+        throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
+            String.format("There is already an active job named %s with id: %s. If you want "
+                + "to submit a second job, try again by setting a different name using --jobName.",
+                newJob.getName(), jobResult.getId()));
+      }
+    }
+
+    LOG.info("To access the Dataflow monitoring console, please navigate to {}",
+        MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
+    System.out.println("Submitted job: " + jobResult.getId());
+
+    LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
+        MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
+
+    return dataflowPipelineJob;
+  }
+
+  /**
+   * Returns the DataflowPipelineTranslator associated with this object.
+   */
+  public DataflowPipelineTranslator getTranslator() {
+    return translator;
+  }
+
+  /**
+   * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}.
+   */
+  @Experimental
+  public void setHooks(DataflowPipelineRunnerHooks hooks) {
+    this.hooks = hooks;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /** Outputs a warning about PCollection views without deterministic key coders. */
+  private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
+    // We need to wait till this point to determine the names of the transforms since only
+    // at this time do we know the hierarchy of the transforms otherwise we could
+    // have just recorded the full names during apply time.
+    if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
+      final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
+      pipeline.traverseTopologically(new PipelineVisitor() {
+        @Override
+        public void visitValue(PValue value, TransformTreeNode producer) {
+        }
+
+        @Override
+        public void visitTransform(TransformTreeNode node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+        }
+
+        @Override
+        public void enterCompositeTransform(TransformTreeNode node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+        }
+
+        @Override
+        public void leaveCompositeTransform(TransformTreeNode node) {
+        }
+      });
+
+      LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
+          + "because the key coder is not deterministic. Falling back to singleton implementation "
+          + "which may cause memory and/or performance problems. Future major versions of "
+          + "Dataflow will require deterministic key coders.",
+          ptransformViewNamesWithNonDeterministicKeyCoders);
+    }
+  }
+
+  /**
+   * Returns true if the passed in {@link PCollection} needs to be materialiazed using
+   * an indexed format.
+   */
+  boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
+    return pcollectionsRequiringIndexedFormat.contains(pcol);
+  }
+
+  /**
+   * Marks the passed in {@link PCollection} as requiring to be materialized using
+   * an indexed format.
+   */
+  private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
+    pcollectionsRequiringIndexedFormat.add(pcol);
+  }
+
+  /** A set of {@link View}s with non-deterministic key coders. */
+  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
+
+  /**
+   * Records that the {@link PTransform} requires a deterministic key coder.
+   */
+  private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
+    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
+  }
+
+  /**
+   * A {@link GroupByKey} transform for the {@link DataflowPipelineRunner} which sorts
+   * values using the secondary key {@code K2}.
+   *
+   * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
+   * the empty window. Care must be taken *afterwards* to either re-window
+   * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
+   * values being within a window.
+   */
+  static class GroupByKeyAndSortValuesOnly<K1, K2, V>
+      extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
+    private GroupByKeyAndSortValuesOnly() {
+    }
+
+    @Override
+    public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
+      PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
+          PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
+          input.getPipeline(),
+          WindowingStrategy.globalDefault(),
+          IsBounded.BOUNDED);
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
+      rval.setCoder(
+          KvCoder.of(inputCoder.getKeyCoder(),
+          IterableCoder.of(inputCoder.getValueCoder())));
+      return rval;
+    }
+  }
+
+  /**
+   * A {@link PTransform} that groups the values by a hash of the window's byte representation
+   * and sorts the values using the windows byte representation.
+   */
+  private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
+      PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
+
+    /**
+     * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
+     * grouping by the hash of the window's byte representation and sorting the grouped values
+     * using the window's byte representation.
+     */
+    @SystemDoFnInternal
+    private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
+        extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
+
+      private final IsmRecordCoder<?> ismCoderForHash;
+      private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
+        this.ismCoderForHash = ismCoderForHash;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        @SuppressWarnings("unchecked")
+        W window = (W) c.window();
+        c.output(
+            KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
+                KV.of(window,
+                    WindowedValue.of(
+                        c.element(),
+                        c.timestamp(),
+                        c.window(),
+                        c.pane()))));
+      }
+    }
+
+    private final IsmRecordCoder<?> ismCoderForHash;
+    private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
+      this.ismCoderForHash = ismCoderForHash;
+    }
+
+    @Override
+    public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+      PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
+          input.apply(ParDo.of(
+              new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
+      rval.setCoder(
+          KvCoder.of(
+              VarIntCoder.of(),
+              KvCoder.of(windowCoder,
+                  FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
+      return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
+   * byte representation and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsSingleton<T>
+      extends PTransform<PCollection<T>, PCollectionView<T>> {
+
+    /**
+     * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
+     * <ul>
+     *   <li>Key 1: Window
+     *   <li>Value: Windowed value
+     * </ul>
+     */
+    static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                     IsmRecord<WindowedValue<T>>> {
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
+        while (iterator.hasNext()) {
+          KV<W, WindowedValue<T>> next = iterator.next();
+          c.output(
+              IsmRecord.of(
+                  ImmutableList.of(next.getKey()), next.getValue()));
+        }
+      }
+    }
+
+    private final DataflowPipelineRunner runner;
+    private final View.AsSingleton<T> transform;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+    public BatchViewAsSingleton(DataflowPipelineRunner runner, View.AsSingleton<T> transform) {
+      this.runner = runner;
+      this.transform = transform;
+    }
+
+    @Override
+    public PCollectionView<T> apply(PCollection<T> input) {
+      return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
+          runner,
+          input,
+          new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(),
+          transform.hasDefaultValue(),
+          transform.defaultValue(),
+          input.getCoder());
+    }
+
+    static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
+        applyForSingleton(
+            DataflowPipelineRunner runner,
+            PCollection<T> input,
+            DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                 IsmRecord<WindowedValue<FinalT>>> doFn,
+            boolean hasDefault,
+            FinalT defaultValue,
+            Coder<FinalT> defaultValueCoder) {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      PCollectionView<ViewT> view =
+          (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
+              input.getPipeline(),
+              (WindowingStrategy) input.getWindowingStrategy(),
+              hasDefault,
+              defaultValue,
+              defaultValueCoder);
+
+      IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
+          coderForSingleton(windowCoder, defaultValueCoder);
+
+      PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
+              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+              .apply(ParDo.of(doFn));
+      reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+      return reifiedPerWindowAndSorted.apply(
+          CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsSingleton";
+    }
+
+    static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
+        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+      return IsmRecordCoder.of(
+          1, // We hash using only the window
+          0, // There are no metadata records
+          ImmutableList.<Coder<?>>of(windowCoder),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsIterable View.AsIterable} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
+   * and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Key 2: Index offset within window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsIterable<T>
+      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
+
+    private final DataflowPipelineRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+    public BatchViewAsIterable(DataflowPipelineRunner runner, View.AsIterable<T> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
+      PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
+          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+      return BatchViewAsList.applyForIterableLike(runner, input, view);
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsList View.AsList} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
+   * and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Key 2: Index offset within window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsList<T>
+      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
+     * global window. Each {@link IsmRecord} has
+     * <ul>
+     *   <li>Key 1: Global window</li>
+     *   <li>Key 2: Index offset within window</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     */
+    @SystemDoFnInternal
+    static class ToIsmRecordForGlobalWindowDoFn<T>
+        extends DoFn<T, IsmRecord<WindowedValue<T>>> {
+
+      long indexInBundle;
+      @Override
+      public void startBundle(Context c) throws Exception {
+        indexInBundle = 0;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(IsmRecord.of(
+            ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
+            WindowedValue.of(
+                c.element(),
+                c.timestamp(),
+                GlobalWindow.INSTANCE,
+                c.pane())));
+        indexInBundle += 1;
+      }
+    }
+
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+     * to locate the window boundaries. The {@link IsmRecord} has:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Key 2: Index offset within window</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     */
+    @SystemDoFnInternal
+    static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                     IsmRecord<WindowedValue<T>>> {
+
+      private final Coder<W> windowCoder;
+      ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        long elementsInWindow = 0;
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
+          // Compare to see if this is a new window so we can reset the index counter i
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Reset i since we have a new window.
+            elementsInWindow = 0;
+          }
+          c.output(IsmRecord.of(
+              ImmutableList.of(value.getKey(), elementsInWindow),
+              value.getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          elementsInWindow += 1;
+        }
+      }
+    }
+
+    private final DataflowPipelineRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+    public BatchViewAsList(DataflowPipelineRunner runner, View.AsList<T> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<List<T>> apply(PCollection<T> input) {
+      PCollectionView<List<T>> view = PCollectionViews.listView(
+          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+      return applyForIterableLike(runner, input, view);
+    }
+
+    static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
+        DataflowPipelineRunner runner,
+        PCollection<T> input,
+        PCollectionView<ViewT> view) {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
+
+      // If we are working in the global window, we do not need to do a GBK using the window
+      // as the key since all the elements of the input PCollection are already such.
+      // We just reify the windowed value while converting them to IsmRecords and generating
+      // an index based upon where we are within the bundle. Each bundle
+      // maps to one file exactly.
+      if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
+        PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
+            input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
+        reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+        runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+        return reifiedPerWindowAndSorted.apply(
+            CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+      }
+
+      PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
+              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+              .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
+      reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+      return reifiedPerWindowAndSorted.apply(
+          CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsList";
+    }
+
+    static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
+        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+      // TODO: swap to use a variable length long coder which has values which compare
+      // the same as their byte representation compare lexicographically within the key coder
+      return IsmRecordCoder.of(
+          1, // We hash using only the window
+          0, // There are no metadata records
+          ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMap View.AsMap} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+   * representation. Each record is structured as follows:
+   * <ul>
+   *   <li>Key 1: User key K</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: 0L (constant)</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   *
+   * <p>Alongside the data records, there are the following metadata records:
+   * <ul>
+   *   <li>Key 1: Metadata Key</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index [0, size of map]</li>
+   *   <li>Value: variable length long byte representation of size of map if index is 0,
+   *              otherwise the byte representation of a key</li>
+   * </ul>
+   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
+   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+   * {@code [1, size of map]}.
+   *
+   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * a warning to users to specify a deterministic key coder.
+   */
+  static class BatchViewAsMap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
+
+    /**
+     * A {@link DoFn} which groups elements by window boundaries. For each group,
+     * the group of elements is transformed into a {@link TransformedMap}.
+     * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
+     * and contains a function {@code WindowedValue<V> -> V}.
+     *
+     * <p>Outputs {@link IsmRecord}s having:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Value: Transformed map containing a transform that removes the encapsulation
+     *              of the window around each value,
+     *              {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
+     * </ul>
+     */
+    static class ToMapDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+                     IsmRecord<WindowedValue<TransformedMap<K,
+                                             WindowedValue<V>,
+                                             V>>>> {
+
+      private final Coder<W> windowCoder;
+      ToMapDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c)
+          throws Exception {
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        Optional<W> previousWindow = Optional.absent();
+        Map<K, WindowedValue<V>> map = new HashMap<>();
+        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Construct the transformed map containing all the elements since we
+            // are at a window boundary.
+            c.output(IsmRecord.of(
+                ImmutableList.of(previousWindow.get()),
+                valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+            map = new HashMap<>();
+          }
+
+          // Verify that the user isn't trying to insert the same key multiple times.
+          checkState(!map.containsKey(kv.getValue().getValue().getKey()),
+              "Multiple values [%s, %s] found for single key [%s] within window [%s].",
+              map.get(kv.getValue().getValue().getKey()),
+              kv.getValue().getValue().getValue(),
+              kv.getKey());
+          map.put(kv.getValue().getValue().getKey(),
+                  kv.getValue().withValue(kv.getValue().getValue().getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          previousWindow = Optional.of(kv.getKey());
+        }
+
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a transformed map containing all the elements since the last
+        // window boundary.
+        c.output(IsmRecord.of(
+            ImmutableList.of(previousWindow.get()),
+            valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+      }
+    }
+
+    private final DataflowPipelineRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+    public BatchViewAsMap(DataflowPipelineRunner runner, View.AsMap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
+      return this.<BoundedWindow>applyInternal(input);
+    }
+
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+        applyInternal(PCollection<KV<K, V>> input) {
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
+            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+        return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
+      } catch (NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+        // Since the key coder is not deterministic, we convert the map into a singleton
+        // and return a singleton view equivalent.
+        return applyForSingletonFallback(input);
+      }
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsMap";
+    }
+
+    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+        applyForSingletonFallback(PCollection<KV<K, V>> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<Function<WindowedValue<V>, V>> transformCoder =
+          (Coder) SerializableCoder.of(WindowedValueToValue.class);
+
+      Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
+          TransformedMapCoder.of(
+          transformCoder,
+          MapCoder.of(
+              inputCoder.getKeyCoder(),
+              FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
+
+      TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
+          WindowedValueToValue.<V>of(),
+          ImmutableMap.<K, WindowedValue<V>>of());
+
+      return BatchViewAsSingleton.<KV<K, V>,
+                                   TransformedMap<K, WindowedValue<V>, V>,
+                                   Map<K, V>,
+                                   W> applyForSingleton(
+          runner,
+          input,
+          new ToMapDoFn<K, V, W>(windowCoder),
+          true,
+          defaultValue,
+          finalValueCoder);
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMultimap View.AsMultimap} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+   * representation. Each record is structured as follows:
+   * <ul>
+   *   <li>Key 1: User key K</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index offset for a given key and window.</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   *
+   * <p>Alongside the data records, there are the following metadata records:
+   * <ul>
+   *   <li>Key 1: Metadata Key</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index [0, size of map]</li>
+   *   <li>Value: variable length long byte representation of size of map if index is 0,
+   *              otherwise the byte representation of a key</li>
+   * </ul>
+   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
+   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+   * {@code [1, size of map]}.
+   *
+   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * a warning to users to specify a deterministic key coder.
+   */
+  static class BatchViewAsMultimap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
+    /**
+     * A {@link PTransform} that groups elements by the hash of window's byte representation
+     * if the input {@link PCollection} is not within the global window. Otherwise by the hash
+     * of the window and key's byte representation. This {@link PTransform} also sorts
+     * the values by the combination of the window and key's byte representations.
+     */
+    private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
+        extends PTransform<PCollection<KV<K, V>>,
+                           PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
+
+      @SystemDoFnInternal
+      private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
+          extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
+          implements DoFn.RequiresWindowAccess {
+
+        private final IsmRecordCoder<?> coder;
+        private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
+          this.coder = coder;
+        }
+
+        @Override
+        public void processElement(ProcessContext c) throws Exception {
+          @SuppressWarnings("unchecked")
+          W window = (W) c.window();
+
+          c.output(
+              KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
+                  KV.of(KV.of(c.element().getKey(), window),
+                      WindowedValue.of(
+                          c.element().getValue(),
+                          c.timestamp(),
+                          (BoundedWindow) window,
+                          c.pane()))));
+        }
+      }
+
+      private final IsmRecordCoder<?> coder;
+      public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
+        this.coder = coder;
+      }
+
+      @Override
+      public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
+          apply(PCollection<KV<K, V>> input) {
+
+        @SuppressWarnings("unchecked")
+        Coder<W> windowCoder = (Coder<W>)
+            input.getWindowingStrategy().getWindowFn().windowCoder();
+        @SuppressWarnings("unchecked")
+        KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+
+        PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
+        keyedByHash = input.apply(
+            ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
+        keyedByHash.setCoder(
+            KvCoder.of(
+                VarIntCoder.of(),
+                KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
+                    FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+        return keyedByHash.apply(
+            new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
+      }
+    }
+
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+     * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Key 2: User key K</li>
+     *   <li>Key 3: Index offset for a given key and window.</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     *
+     * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
+     * and the unique key count per window to {@code outputForSize}.
+     *
+     * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
+     * throw an {@link IllegalStateException} if more than one key per window is found.
+     */
+    static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
+                     IsmRecord<WindowedValue<V>>> {
+
+      private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
+      private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
+      private final Coder<W> windowCoder;
+      private final Coder<K> keyCoder;
+      private final IsmRecordCoder<WindowedValue<V>> ismCoder;
+      private final boolean uniqueKeysExpected;
+      ToIsmRecordForMapLikeDoFn(
+          TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
+          TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
+          Coder<W> windowCoder,
+          Coder<K> keyCoder,
+          IsmRecordCoder<WindowedValue<V>> ismCoder,
+          boolean uniqueKeysExpected) {
+        this.outputForSize = outputForSize;
+        this.outputForEntrySet = outputForEntrySet;
+        this.windowCoder = windowCoder;
+        this.keyCoder = keyCoder;
+        this.ismCoder = ismCoder;
+        this.uniqueKeysExpected = uniqueKeysExpected;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        long currentKeyIndex = 0;
+        // We use one based indexing while counting
+        long currentUniqueKeyCounter = 1;
+        Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
+
+        KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
+        Object currentKeyStructuralValue =
+            keyCoder.structuralValue(currentValue.getKey().getKey());
+        Object currentWindowStructuralValue =
+            windowCoder.structuralValue(currentValue.getKey().getValue());
+
+        while (iterator.hasNext()) {
+          KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
+          Object nextKeyStructuralValue =
+              keyCoder.structuralValue(nextValue.getKey().getKey());
+          Object nextWindowStructuralValue =
+              windowCoder.structuralValue(nextValue.getKey().getValue());
+
+          outputDataRecord(c, currentValue, currentKeyIndex);
+
+          final long nextKeyIndex;
+          final long nextUniqueKeyCounter;
+
+          // Check to see if its a new window
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            // The next value is a new window, so we output for size the number of unique keys
+            // seen and the last key of the window. We also reset the next key index the unique
+            // key counter.
+            outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+            outputMetadataRecordForEntrySet(c, currentValue);
+
+            nextKeyIndex = 0;
+            nextUniqueKeyCounter = 1;
+          } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
+            // It is a new key within the same window so output the key for the entry set,
+            // reset the key index and increase the count of unique keys seen within this window.
+            outputMetadataRecordForEntrySet(c, currentValue);
+
+            nextKeyIndex = 0;
+            nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
+          } else if (!uniqueKeysExpected) {
+            // It is not a new key so we don't have to output the number of elements in this
+            // window or increase the unique key counter. All we do is increase the key index.
+
+            nextKeyIndex = currentKeyIndex + 1;
+            nextUniqueKeyCounter = currentUniqueKeyCounter;
+          } else {
+            throw new IllegalStateException(String.format(
+                "Unique keys are expected but found key %s with values %s and %s in window %s.",
+                currentValue.getKey().getKey(),
+                currentValue.getValue().getValue(),
+                nextValue.getValue().getValue(),
+                currentValue.getKey().getValue()));
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+          currentKeyStructuralValue = nextKeyStructuralValue;
+          currentKeyIndex = nextKeyIndex;
+          currentUniqueKeyCounter = nextUniqueKeyCounter;
+        }
+
+        outputDataRecord(c, currentValue, currentKeyIndex);
+        outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a record with the number of unique keys seen.
+        outputMetadataRecordForEntrySet(c, currentValue);
+      }
+
+      /** This outputs the data record. */
+      private void outputDataRecord(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
+        IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
+            ImmutableList.of(
+                value.getKey().getKey(),
+                value.getKey().getValue(),
+                keyIndex),
+            value.getValue());
+        c.output(ismRecord);
+      }
+
+      /**
+       * This outputs records which will be used to compute the number of keys for a given window.
+       */
+      private void outputMetadataRecordForSize(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
+        c.sideOutput(outputForSize,
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+                                                 value.getKey().getValue())),
+                KV.of(value.getKey().getValue(), uniqueKeyCount)));
+      }
+
+      /** This outputs records which will be used to construct the entry set. */
+      private void outputMetadataRecordForEntrySet(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
+        c.sideOutput(outputForEntrySet,
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+                                                 value.getKey().getValue())),
+                KV.of(value.getKey().getValue(), value.getKey().getKey())));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
+       * <ul>
+       *   <li>Key 1: META key</li>
+       *   <li>Key 2: window</li>
+       *   <li>Key 3: 0L (constant)</li>
+       *   <li>Value: sum of values for window</li>
+       * </ul>
+       *
+       * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
+       * per window for map and multimap side inputs.
+       */
+    static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
+      private final Coder<W> windowCoder;
+      ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
+        KV<W, Long> currentValue = iterator.next();
+        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+        long size = 0;
+        while (iterator.hasNext()) {
+          KV<W, Long> nextValue = iterator.next();
+          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+          size += currentValue.getValue();
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            c.output(IsmRecord.<WindowedValue<V>>meta(
+                ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+                CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+            size = 0;
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+        }
+
+        size += currentValue.getValue();
+        // Output the final value since it is guaranteed to be on a window boundary.
+        c.output(IsmRecord.<WindowedValue<V>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
+       * <ul>
+       *   <li>Key 1: META key</li>
+       *   <li>Key 2: window</li>
+       *   <li>Key 3: index offset (1-based index)</li>
+       *   <li>Value: key</li>
+       * </ul>
+       *
+       * <p>This {@link DoFn} is meant to be used to output index to key records
+       * per window for map and multimap side inputs.
+       */
+    static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
+
+      private final Coder<K> keyCoder;
+      private final Coder<W> windowCoder;
+      ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
+        this.keyCoder = keyCoder;
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
+        KV<W, K> currentValue = iterator.next();
+        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+        long elementsInWindow = 1;
+        while (iterator.hasNext()) {
+          KV<W, K> nextValue = iterator.next();
+          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+          c.output(IsmRecord.<WindowedValue<V>>meta(
+              ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+              CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+          elementsInWindow += 1;
+
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            elementsInWindow = 1;
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+        }
+
+        // Output the final value since it is guaranteed to be on a window boundary.
+        c.output(IsmRecord.<WindowedValue<V>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+            CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
+     * partition, the set of elements is transformed into a {@link TransformedMap}.
+     * The transformed {@code Map<K, Iterable<V>>} is backed by a
+     * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
+     * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
+     *
+     * <p>Outputs {@link IsmRecord}s having:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Value: Transformed map containing a transform that removes the encapsulation
+     *              of the window around each value,
+     *              {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
+     * </ul>
+     */
+    static class ToMultimapDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+                     IsmRecord<WindowedValue<TransformedMap<K,
+                                                            Iterable<WindowedValue<V>>,
+                                                            Iterable<V>>>>> {
+
+      private final Coder<W> windowCoder;
+      ToMultimapDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c)
+          throws Exception {
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        Optional<W> previousWindow = Optional.absent();
+        Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
+        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Construct the transformed map containing all the elements since we
+            // are at a window boundary.
+            @SuppressWarnings({"unchecked", "rawtypes"})
+            Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+            c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+                                                             Iterable<WindowedValue<V>>,
+                                                             Iterable<V>>>>of(
+                ImmutableList.of(previousWindow.get()),
+                valueInEmptyWindows(
+                    new TransformedMap<>(
+                        IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+            multimap = HashMultimap.create();
+          }
+
+          multimap.put(kv.getValue().getValue().getKey(),
+                       kv.getValue().withValue(kv.getValue().getValue().getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          previousWindow = Optional.of(kv.getKey());
+        }
+
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a transformed map containing all the elements since the last
+        // window boundary.
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+        c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+                                                         Iterable<WindowedValue<V>>,
+                                                         Iterable<V>>>>of(
+            ImmutableList.of(previousWindow.get()),
+            valueInEmptyWindows(
+                new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+      }
+    }
+
+    private final DataflowPipelineRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+    public BatchViewAsMultimap(DataflowPipelineRunner runner, View.AsMultimap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+      return this.<BoundedWindow>applyInternal(input);
+    }
+
+    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+        applyInternal(PCollection<KV<K, V>> input) {
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
+            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+
+        return applyForMapLike(runner, input, view, false /* unique keys not expected */);
+      } catch (NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+        // Since the key coder is not deterministic, we convert the map into a singleton
+        // and return a singleton view equivalent.
+        return applyForSingletonFallback(input);
+      }
+    }
+
+    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+        applyForSingletonFallback(PCollection<KV<K, V>> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
+          (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
+
+      Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
+          TransformedMapCoder.of(
+          transformCoder,
+          MapCoder.of(
+              inputCoder.getKeyCoder(),
+              IterableCoder.of(
+                  FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+      TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
+          new TransformedMap<>(
+              IterableWithWindowedValuesToIterable.<V>of(),
+              ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
+
+      return BatchViewAsSingleton.<KV<K, V>,
+                                   TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
+                                   Map<K, Iterable<V>>,
+                                   W> applyForSingleton(
+          runner,
+          input,
+          new ToMultimapDoFn<K, V, W>(windowCoder),
+          true,
+          defaultValue,
+          finalValueCoder);
+    }
+
+    private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
+        DataflowPipelineRunner runner,
+        PCollection<KV<K, V>> input,
+        PCollectionView<ViewT> view,
+        boolean uniqueKeysExpected) throws NonDeterministicException {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      // If our key coder is deterministic, we can use the key portion of each KV
+      // part of a composite key containing the window , key and index.
+      inputCoder.getKeyCoder().verifyDeterministic();
+
+      IsmRecordCoder<WindowedValue<V>> ismCoder =
+          coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+
+      // Create the various output tags representing the main output containing the data stream
+      // and the side outputs containing the metadata about the size and entry set.
+      TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
+      TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
+      TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
+
+      // Process all the elements grouped by key hash, and sorted by key and then window
+      // outputting to all the outputs defined above.
+      PCollectionTuple outputTuple = input
+           .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
+           .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
+                   outputForSizeTag, outputForEntrySetTag,
+                   windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
+                       .withOutputTags(mainOutputTag,
+                                       TupleTagList.of(
+                                           ImmutableList.<TupleTag<?>>of(outputForSizeTag,
+                                                                         outputForEntrySetTag))));
+
+      // Set the coder on the main data output.
+      PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
+          outputTuple.get(mainOutputTag);
+      perHashWithReifiedWindows.setCoder(ismCoder);
+
+      // Set the coder on the metadata output for size and process the entries
+      // producing a [META, Window, 0L] record per window storing the number of unique keys
+      // for each window.
+      PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
+      outputForSize.setCoder(
+          KvCoder.of(VarIntCoder.of(),
+                     KvCoder.of(windowCoder, VarLongCoder.of())));
+      PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
+          .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
+          .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
+      windowMapSizeMetadata.setCoder(ismCoder);
+
+      // Set the coder on the metadata output destined to build the entry set and process the
+      // entries producing a [META, Window, Index] record per window key pair storing the key.
+      PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
+          outputTuple.get(outputForEntrySetTag);
+      outputForEntrySet.setCoder(
+          KvCoder.of(VarIntCoder.of(),
+                     KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
+      PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
+          .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
+          .apply(ParDo.of(
+              new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
+      windowMapKeysMetadata.setCoder(ismCoder);
+
+      // Set that all these outputs should be materialized using an indexed format.
+      runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
+      runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
+      runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
+
+      PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
+          PCollectionList.of(ImmutableList.of(
+              perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
+
+      return Pipeline.applyTransform(outputs,
+                                     Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
+          .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
+                                        ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsMultimap";
+    }
+
+    static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
+        Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
+      // TODO: swap to use a variable length long coder which has values which compare
+      // the same as their byte representation compare lexicographically within the key coder
+      return IsmRecordCoder.of(
+          1, // We use only the key for hashing when producing value records
+          2, // Since the key is not present, we add the window to the hash when
+             // producing metadata records
+          ImmutableList.of(
+              MetadataKeyCoder.of(keyCoder),
+              windowCoder,
+              BigEndianLongCoder.of()),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
+   * {@code V1 -> V2}.
+   */
+  static class TransformedMap<K, V1, V2>
+      extends ForwardingMap<K, V2> {
+    private final Function<V1, V2> transform;
+    private final Map<K, V1> originalMap;
+    private final Map<K, V2> transformedMap;
+
+    private TransformedMap(Function<V1, V2> transform, Map<K, V1> originalMap) {
+      this.transform = transform;
+      this.originalMap = Collections.unmodifiableMap(originalMap);
+      this.transformedMap = Maps.transformValues(originalMap, transform);
+    }
+
+    @Override
+    protected Map<K, V2> delegate() {
+      return transformedMap;
+    }
+  }
+
+  /**
+   * A {@link Coder} for {@link TransformedMap}s.
+   */
+  static class TransformedMapCoder<K, V1, V2>
+      extends StandardCoder<TransformedMap<K, V1, V2>> {
+    private final Coder<Function<V1, V2>> transformCoder;
+    private final Coder<Map<K, V1>> originalMapCoder;
+
+    private TransformedMapCoder(
+        Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
+      this.transformCoder = transformCoder;
+      this.originalMapCoder = originalMapCoder;
+    }
+
+    public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
+        Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
+      return new TransformedMapCoder<>(transformCoder, originalMapCoder);
+    }
+
+    @JsonCreator
+    public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
+        @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+        List<Coder<?>> components) {
+      checkArgument(components.size() == 2,
+          "Expecting 2 components, got " + components.size());
+      @SuppressWarnings("unchecked")
+      Coder<Function<V1, V2>> transformCoder = (Coder<Function<V1, V2>>) components.get(0);
+      @SuppressWarnings("unchecked")
+      Coder<Map<K, V1>> originalMapCoder = (Coder<Map<K, V1>>) components.get(1);
+      return of(transformCoder, originalMapCoder);
+    }
+
+    @Override
+    public void encode(TransformedMap<K, V1, V2> value, OutputStream outStream,
+        Coder.Context contex

<TRUNCATED>


[55/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
index 8438fff..e9e9e36 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessPipelineResult;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.InProcessPipelineResult;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
index a80ae93..9922413 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.equalTo;
@@ -24,24 +24,25 @@ import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.doAnswer;
 
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Mean;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
-import com.google.cloud.dataflow.sdk.util.PCollectionViews;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Mean;
+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.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+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.PCollectionView;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
index 85ca140..b496981 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespaces;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
index c75e3be..b89340e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
@@ -15,27 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.Keys;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableSet;
 
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
index df3adcf..152cac4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkArgument;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
index 87247ac..a048e3a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
@@ -25,35 +25,35 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.BagState;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.WatermarkHoldState;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+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.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
@@ -119,7 +119,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoMultiEvaluatorFactory()
             .forApplication(
                 mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
@@ -204,7 +204,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoMultiEvaluatorFactory()
             .forApplication(
                 mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
@@ -294,7 +294,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoMultiEvaluatorFactory()
             .forApplication(
                 mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
@@ -404,7 +404,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoMultiEvaluatorFactory()
             .forApplication(
                 mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
index ddb58c2..dfd857e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
@@ -25,32 +25,32 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.BagState;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import com.google.cloud.dataflow.sdk.util.state.StateTag;
-import com.google.cloud.dataflow.sdk.util.state.StateTags;
-import com.google.cloud.dataflow.sdk.util.state.WatermarkHoldState;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+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.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
@@ -95,7 +95,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoSingleEvaluatorFactory()
             .forApplication(
                 collection.getProducingTransformInternal(), inputBundle, evaluationContext);
@@ -211,7 +211,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CounterSet counters = new CounterSet();
     when(evaluationContext.createCounterSet()).thenReturn(counters);
 
-    com.google.cloud.dataflow.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
+    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
         new ParDoSingleEvaluatorFactory()
             .forApplication(
                 mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
index 4c08777..239ce27 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.theInstance;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.TextIOTest;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
+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.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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
index c4a5376..33dbbdc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.any;
 import static org.hamcrest.Matchers.is;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
index 7e87515..d29ffac 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -27,17 +27,18 @@ import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.util.concurrent.MoreExecutors;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
index dfcafaa..82657c0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -25,24 +25,25 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.io.CountingSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;
@@ -319,12 +320,12 @@ public class UnboundedReadEvaluatorFactoryTest {
       public void encode(
           TestCheckpointMark value,
           OutputStream outStream,
-          com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+          org.apache.beam.sdk.coders.Coder.Context context)
           throws CoderException, IOException {}
 
       @Override
       public TestCheckpointMark decode(
-          InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context)
+          InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
           throws CoderException, IOException {
         return new TestCheckpointMark();
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
index ce40a83..05346dc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.nullValue;
@@ -23,21 +23,22 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.util.PCollectionViews;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
index cba1f31..3b36bc5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
index 2b57f33..d41825d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
@@ -15,28 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
index 123fe57..5b5274f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
 import com.google.common.base.Strings;
 
 import org.hamcrest.CoreMatchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/DataflowJUnitTestRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/DataflowJUnitTestRunner.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/DataflowJUnitTestRunner.java
index f0f341a..457712f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/DataflowJUnitTestRunner.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/DataflowJUnitTestRunner.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
+
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
 
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.reflect.ClassPath;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
index 6c4ecf0..caf8fd7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.junit.Assert.fail;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
index cff2df7..1d7e18a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
-import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
+import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java
index baf267a..a97726b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import com.google.api.client.util.NanoClock;
 import com.google.api.client.util.Sleeper;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java
index f012764..9d15c0f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 a1f88fe..f540948 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
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
+
+import static org.apache.beam.sdk.testing.SerializableMatchers.anything;
+import static org.apache.beam.sdk.testing.SerializableMatchers.not;
 
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.anything;
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.not;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+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.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java
index 899c374..427f2da 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValueBase;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.sdk.testing;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValueBase;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.Iterables;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProvider.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProvider.java
index b190f5f..decea32 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProvider.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProvider.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import org.joda.time.DateTimeUtils;
 import org.joda.time.format.ISODateTimeFormat;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProviderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProviderTest.java
index 878b828..f81cbd2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProviderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ResetDateTimeProviderTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
index cd1b29a..423026d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import org.junit.rules.ExternalResource;
 import org.junit.rules.TestRule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
index 14429da..1a36586 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
index 0370305..7cbd4a8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
@@ -15,20 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
+
+import static org.apache.beam.sdk.testing.SerializableMatchers.allOf;
+import static org.apache.beam.sdk.testing.SerializableMatchers.anything;
+import static org.apache.beam.sdk.testing.SerializableMatchers.containsInAnyOrder;
+import static org.apache.beam.sdk.testing.SerializableMatchers.kvWithKey;
+import static org.apache.beam.sdk.testing.SerializableMatchers.not;
 
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.allOf;
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.anything;
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.containsInAnyOrder;
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.kvWithKey;
-import static com.google.cloud.dataflow.sdk.testing.SerializableMatchers.not;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.KV;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java
index c3b1797..254a1d5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import com.google.api.client.util.Sleeper;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java
index c782adf..623224d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
+
+import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
 
-import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
 import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
index 147cc49..fec0b48 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
index bd6c8a2..6d62e08 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.ApproximateQuantiles.ApproximateQuantilesCombineFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.runners.DirectPipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ApproximateQuantiles.ApproximateQuantilesCombineFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.CoreMatchers;
 import org.hamcrest.Description;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
index b6cd697..3a4b813 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.TestUtils;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.TestUtils;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
index 7d3aedf..0a02538 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
@@ -15,30 +15,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.NullableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Combine.BinaryCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFns.CoCombineResult;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn;
-import com.google.cloud.dataflow.sdk.transforms.Min.MinIntegerFn;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Combine.BinaryCombineFn;
+import org.apache.beam.sdk.transforms.CombineFns.CoCombineResult;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.Max.MaxIntegerFn;
+import org.apache.beam.sdk.transforms.Min.MinIntegerFn;
+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.TupleTag;
+
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 fce86fd..049baa3 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
@@ -15,47 +15,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.DoubleCoder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunners;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.util.PerKeyCombineFnRunners;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+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.MoreObjects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
index 1a5ca8b..ca898b3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.NO_LINES;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
index 432795d..393fede 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
@@ -15,24 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.LINES;
+import static org.apache.beam.sdk.TestUtils.LINES_ARRAY;
+import static org.apache.beam.sdk.TestUtils.NO_LINES;
+import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES;
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnContextTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnContextTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnContextTest.java
index 6eee7d5..2e588c7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnContextTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnContextTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
index a286bd6..bf9899c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.anyLong;
@@ -23,8 +23,8 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.DelegatingAggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn.DelegatingAggregator;
 
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java
index 5cd69fc..7399322 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ExtraContextFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ProcessContext;
-import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ProcessElement;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.transforms.DoFnWithContext.Context;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ExtraContextFactory;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ProcessContext;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ProcessElement;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingInternals;
 
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
index 9de7ce7..b7906b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.CoreMatchers.isA;
 import static org.hamcrest.Matchers.empty;
@@ -23,11 +23,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Max.MaxIntegerFn;
+import org.apache.beam.sdk.transforms.display.DisplayData;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
index b00681a..4df5be7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.CoreMatchers.hasItems;
 import static org.hamcrest.Matchers.equalTo;
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester.OutputElementWithTimestamp;
+import org.apache.beam.sdk.transforms.DoFnTester.OutputElementWithTimestamp;
 
 import org.joda.time.Instant;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnWithContextTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnWithContextTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnWithContextTest.java
index 488bcfc..40c80b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnWithContextTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnWithContextTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.CoreMatchers.isA;
 import static org.junit.Assert.assertEquals;
@@ -25,10 +25,10 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Max.MaxIntegerFn;
 
 import org.junit.Rule;
 import org.junit.Test;



[64/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ed25926..b95fe15 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
@@ -15,23 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+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.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
@@ -310,20 +311,20 @@ public class TextIO {
         final Bounded<T> read;
         switch(compressionType) {
           case UNCOMPRESSED:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
+            read = org.apache.beam.sdk.io.Read.from(
                 new TextSource<T>(filepattern, coder));
             break;
           case AUTO:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
+            read = org.apache.beam.sdk.io.Read.from(
                 CompressedSource.from(new TextSource<T>(filepattern, coder)));
             break;
           case BZIP2:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
+            read = org.apache.beam.sdk.io.Read.from(
                 CompressedSource.from(new TextSource<T>(filepattern, coder))
                                 .withDecompression(CompressedSource.CompressionMode.BZIP2));
             break;
           case GZIP:
-            read = com.google.cloud.dataflow.sdk.io.Read.from(
+            read = org.apache.beam.sdk.io.Read.from(
                 CompressedSource.from(new TextSource<T>(filepattern, coder))
                                 .withDecompression(CompressedSource.CompressionMode.GZIP));
             break;
@@ -625,7 +626,7 @@ public class TextIO {
         // 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", com.google.cloud.dataflow.sdk.io.Write.to(
+        return input.apply("Write", org.apache.beam.sdk.io.Write.to(
             new TextSink<>(
                 filenamePrefix, filenameSuffix, shardTemplate, coder)));
       }
@@ -769,7 +770,7 @@ public class TextIO {
     }
 
     /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSource.FileBasedReader FileBasedReader}
+     * A {@link org.apache.beam.sdk.io.FileBasedSource.FileBasedReader FileBasedReader}
      * which can decode records delimited by new line characters.
      *
      * See {@link TextSource} for further details.
@@ -944,7 +945,7 @@ public class TextIO {
     }
 
     /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation
+     * A {@link org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation
      * FileBasedWriteOperation} for text files.
      */
     private static class TextWriteOperation<T> extends FileBasedWriteOperation<T> {
@@ -962,7 +963,7 @@ public class TextIO {
     }
 
     /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
+     * A {@link org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
      * for text files.
      */
     private static class TextWriter<T> extends FileBasedWriter<T> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9f4a234..82c8db7 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
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import org.joda.time.Instant;
 
@@ -41,8 +41,8 @@ import javax.annotation.Nullable;
  *   do not guarantee that a given record will only be read a single time.
  * </ul>
  *
- * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} and
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for more information on
+ * <p>See {@link org.apache.beam.sdk.transforms.windowing.Window} and
+ * {@link org.apache.beam.sdk.transforms.windowing.Trigger} for more information on
  * timestamps and watermarks.
  *
  * @param <OutputT> Type of records output by this source.
@@ -98,7 +98,7 @@ public abstract class UnboundedSource<
 
   /**
    * A marker representing the progress and state of an
-   * {@link com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader}.
+   * {@link org.apache.beam.sdk.io.UnboundedSource.UnboundedReader}.
    *
    * <p>For example, this could be offsets in a set of files being read.
    */
@@ -182,7 +182,7 @@ public abstract class UnboundedSource<
      *
      * <p>This can be approximate.  If records are read that violate this guarantee, they will be
      * considered late, which will affect how they will be processed.  See
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} for more information on
+     * {@link org.apache.beam.sdk.transforms.windowing.Window} for more information on
      * late data and how to handle it.
      *
      * <p>However, this value should be as late as possible. Downstream windows may not be able
@@ -193,8 +193,8 @@ public abstract class UnboundedSource<
      * source that does not have natural timestamps, timestamps can be set to the time of
      * reading, in which case the watermark is the current clock time.
      *
-     * <p>See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} and
-     * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for more
+     * <p>See {@link org.apache.beam.sdk.transforms.windowing.Window} and
+     * {@link org.apache.beam.sdk.transforms.windowing.Trigger} for more
      * information on timestamps and watermarks.
      *
      * <p>May be called after {@link #advance} or {@link #start} has returned false, but not before

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 a70105f..a8a2517 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
@@ -15,26 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
+import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.Sink.WriteOperation;
+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.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9ba76c5..4e5a098 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation;
+import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Preconditions;
 
 import java.io.OutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 eae5e8b..1a97db1 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+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 com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.JAXBCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
 import com.google.common.base.Preconditions;
 
 import org.codehaus.stax2.XMLInputFactory2;
@@ -110,7 +111,7 @@ import javax.xml.stream.XMLStreamReader;
  *
  * <p><h3>Permissions</h3>
  * Permission requirements depend on the
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner PipelineRunner} that is
+ * {@link org.apache.beam.sdk.runners.PipelineRunner PipelineRunner} that is
  * used to execute the Dataflow job. Please refer to the documentation of corresponding
  * {@link PipelineRunner PipelineRunners} for more details.
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableIO.java
index 0ce2a5e..01a59e6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableIO.java
@@ -15,37 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.bigtable;
+package org.apache.beam.sdk.io.bigtable;
 
 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;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Proto2Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.Sink.WriteOperation;
+import org.apache.beam.sdk.io.Sink.Writer;
+import org.apache.beam.sdk.io.range.ByteKey;
+import org.apache.beam.sdk.io.range.ByteKeyRange;
+import org.apache.beam.sdk.io.range.ByteKeyRangeTracker;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.DataflowReleaseInfo;
+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.PDone;
 
 import com.google.bigtable.v1.Mutation;
 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.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Proto2Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.io.range.ByteKey;
-import com.google.cloud.dataflow.sdk.io.range.ByteKeyRange;
-import com.google.cloud.dataflow.sdk.io.range.ByteKeyRangeTracker;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.FutureCallback;
@@ -246,7 +246,7 @@ public class BigtableIO {
     public PCollection<Row> apply(PBegin input) {
       BigtableSource source =
           new BigtableSource(getBigtableService(), tableId, filter, ByteKeyRange.ALL_KEYS, null);
-      return input.getPipeline().apply(com.google.cloud.dataflow.sdk.io.Read.from(source));
+      return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source));
     }
 
     @Override
@@ -400,7 +400,7 @@ public class BigtableIO {
     @Override
     public PDone apply(PCollection<KV<ByteString, Iterable<Mutation>>> input) {
       Sink sink = new Sink(tableId, getBigtableService());
-      return input.apply(com.google.cloud.dataflow.sdk.io.Write.to(sink));
+      return input.apply(org.apache.beam.sdk.io.Write.to(sink));
     }
 
     @Override
@@ -797,7 +797,7 @@ public class BigtableIO {
   }
 
   private static class Sink
-      extends com.google.cloud.dataflow.sdk.io.Sink<KV<ByteString, Iterable<Mutation>>> {
+      extends org.apache.beam.sdk.io.Sink<KV<ByteString, Iterable<Mutation>>> {
 
     public Sink(String tableId, BigtableService bigtableService) {
       this.tableId = checkNotNull(tableId, "tableId");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableService.java
index 0c47f65..cfae0ef 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableService.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableService.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.bigtable;
+package org.apache.beam.sdk.io.bigtable;
+
+import org.apache.beam.sdk.io.bigtable.BigtableIO.BigtableSource;
+import org.apache.beam.sdk.values.KV;
 
 import com.google.bigtable.v1.Mutation;
 import com.google.bigtable.v1.Row;
 import com.google.bigtable.v1.SampleRowKeysResponse;
-import com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO.BigtableSource;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Empty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableServiceImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableServiceImpl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableServiceImpl.java
index 9f32022..87651f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableServiceImpl.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/BigtableServiceImpl.java
@@ -15,7 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.bigtable;
+package org.apache.beam.sdk.io.bigtable;
+
+import org.apache.beam.sdk.io.bigtable.BigtableIO.BigtableSource;
+import org.apache.beam.sdk.values.KV;
 
 import com.google.bigtable.admin.table.v1.GetTableRequest;
 import com.google.bigtable.v1.MutateRowRequest;
@@ -30,8 +33,6 @@ import com.google.cloud.bigtable.grpc.BigtableSession;
 import com.google.cloud.bigtable.grpc.async.AsyncExecutor;
 import com.google.cloud.bigtable.grpc.async.HeapSizeManager;
 import com.google.cloud.bigtable.grpc.scanner.ResultScanner;
-import com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO.BigtableSource;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.base.MoreObjects;
 import com.google.common.io.Closer;
 import com.google.common.util.concurrent.ListenableFuture;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/package-info.java
index 553f46c..f094cd4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/bigtable/package-info.java
@@ -18,6 +18,6 @@
 /**
  * Defines transforms for reading and writing from Google Cloud Bigtable.
  *
- * @see com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO
+ * @see org.apache.beam.sdk.io.bigtable.BigtableIO
  */
-package com.google.cloud.dataflow.sdk.io.bigtable;
+package org.apache.beam.sdk.io.bigtable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 5f0050d..c2c0685 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
@@ -17,9 +17,9 @@
  */
 /**
  * Defines transforms for reading and writing common storage formats, including
- * {@link com.google.cloud.dataflow.sdk.io.AvroIO},
- * {@link com.google.cloud.dataflow.sdk.io.BigQueryIO}, and
- * {@link com.google.cloud.dataflow.sdk.io.TextIO}.
+ * {@link org.apache.beam.sdk.io.AvroIO},
+ * {@link org.apache.beam.sdk.io.BigQueryIO}, and
+ * {@link org.apache.beam.sdk.io.TextIO}.
  *
  * <p>The classes in this package provide {@code Read} transforms that create PCollections
  * from existing storage:
@@ -35,4 +35,4 @@
  *                           .to("gs://my_bucket/path/to/numbers"));
  * } </pre>
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java
index 5b9a003..f82a11d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
index c92e7e9..cfd924d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 0d01f8f..cb779fd 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import static com.google.common.base.MoreObjects.toStringHelper;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
index 3cebb77..ea1cf14 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/RangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/RangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/RangeTracker.java
index dad9edc..ad2f119 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/RangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/RangeTracker.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;
 
 /**
  * A {@code RangeTracker} is a thread-safe helper object for implementing dynamic work rebalancing
- * in position-based {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader}
+ * in position-based {@link org.apache.beam.sdk.io.BoundedSource.BoundedReader}
  * subclasses.
  *
  * <h3>Usage of the RangeTracker class hierarchy</h3>
@@ -156,7 +156,7 @@ package com.google.cloud.dataflow.sdk.io.range;
  *
  * <h3>Usage with different models of iteration</h3>
  * When using this class to protect a
- * {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader}, follow the pattern
+ * {@link org.apache.beam.sdk.io.BoundedSource.BoundedReader}, follow the pattern
  * described above.
  *
  * <p>When using this class to protect iteration in the {@code hasNext()/next()}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/package-info.java
index 9117e43..c07a90d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/package-info.java
@@ -19,6 +19,6 @@
  * Provides thread-safe helpers for implementing dynamic work rebalancing in position-based
  * bounded sources.
  *
- * <p>See {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} to get started.
+ * <p>See {@link org.apache.beam.sdk.io.range.RangeTracker} to get started.
  */
-package com.google.cloud.dataflow.sdk.io.range;
+package org.apache.beam.sdk.io.range;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ApplicationNameOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ApplicationNameOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ApplicationNameOptions.java
index 2860fbe..2b78a9f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ApplicationNameOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ApplicationNameOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * Options that allow setting the application name.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
index 5576b5f..db7b69c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/BigQueryOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * Properties needed when using BigQuery with the Dataflow SDK.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Default.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Default.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Default.java
index 843eba1..cd26a87 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Default.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Default.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultValueFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultValueFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultValueFactory.java
index a6e7856..8be5240 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultValueFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DefaultValueFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * An interface used with the {@link Default.InstanceFactory} annotation to specify the class that

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Description.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Description.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Description.java
index 4f90c2a..1622349 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Description.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Description.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
index a505632..718948e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipeline;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.DirectPipeline;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
index 52028cd..4585266 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.util.CredentialFactory;
+import org.apache.beam.sdk.util.GcpCredentialFactory;
+import org.apache.beam.sdk.util.InstanceBuilder;
 
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants;
-import com.google.cloud.dataflow.sdk.util.CredentialFactory;
-import com.google.cloud.dataflow.sdk.util.GcpCredentialFactory;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.io.Files;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
index 130310a..bae4742 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.util.AppEngineEnvironment;
+import org.apache.beam.sdk.util.GcsUtil;
 
-import com.google.cloud.dataflow.sdk.util.AppEngineEnvironment;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
 import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
index 1863141..39cd40e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import com.google.api.client.googleapis.services.AbstractGoogleClient;
 import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Hidden.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Hidden.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Hidden.java
index fd25db8..db64867 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Hidden.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Hidden.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index 2ca845d..17cf5b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
+import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer;
+import org.apache.beam.sdk.options.ProxyInvocationHandler.Serializer;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Context;
 
 import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
-import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Deserializer;
-import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Serializer;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.Context;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.databind.ObjectMapper;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 dac7726..87ac05e 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
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+import org.apache.beam.sdk.util.StringUtils;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
@@ -647,7 +648,7 @@ public class PipelineOptionsFactory {
     }
     out.format("%nUse --help=<OptionsName> for detailed help. For example:%n"
         + "  --help=DataflowPipelineOptions <short names valid for registered options>%n"
-        + "  --help=com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions%n");
+        + "  --help=org.apache.beam.sdk.options.DataflowPipelineOptions%n");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java
index 3fbe7ca..16cf7cd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import com.google.auto.service.AutoService;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9143c0b..4946c59 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
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
 
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Ordering;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9e7c16e..e281625 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
+import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.Registration;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
 import com.google.common.base.Defaults;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
index deb19e9..f84e47b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PubsubOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * Properties that can be set when using Pubsub with the Beam SDK.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java
index 67cb386..a9031cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * Options used to configure streaming.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Validation.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Validation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Validation.java
index b7725a6..458a450 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Validation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/Validation.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/options/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/package-info.java
index 63a03f5..056393a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/package-info.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 /**
- * Defines {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} for
+ * Defines {@link org.apache.beam.sdk.options.PipelineOptions} for
  * configuring pipeline execution.
  *
- * <p>{@link com.google.cloud.dataflow.sdk.options.PipelineOptions} encapsulates the various
+ * <p>{@link org.apache.beam.sdk.options.PipelineOptions} encapsulates the various
  * parameters that describe how a pipeline should be run. {@code PipelineOptions} are created
- * using a {@link com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory}.
+ * using a {@link org.apache.beam.sdk.options.PipelineOptionsFactory}.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
index ab54533..f985a55 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java
@@ -18,17 +18,17 @@
 /**
  * Provides a simple, powerful model for building both batch and
  * streaming parallel data processing
- * {@link com.google.cloud.dataflow.sdk.Pipeline}s.
+ * {@link org.apache.beam.sdk.Pipeline}s.
  *
  * <p>To use the Google Cloud Dataflow SDK, you build a
- * {@link com.google.cloud.dataflow.sdk.Pipeline}, which manages a graph of
- * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s
- * and the {@link com.google.cloud.dataflow.sdk.values.PCollection}s that
+ * {@link org.apache.beam.sdk.Pipeline}, which manages a graph of
+ * {@link org.apache.beam.sdk.transforms.PTransform}s
+ * and the {@link org.apache.beam.sdk.values.PCollection}s that
  * the PTransforms consume and produce.
  *
  * <p>Each Pipeline has a
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to specify
+ * {@link org.apache.beam.sdk.runners.PipelineRunner} to specify
  * where and how it should run after pipeline construction is complete.
  *
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractor.java
index 3008c6c..86a851f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorPipelineExtractor.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AggregatorRetriever;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PValue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AggregatorRetriever;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PValue;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.SetMultimap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorRetrievalException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorRetrievalException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorRetrievalException.java
index 2ed0afa..a0973c3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorRetrievalException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorRetrievalException.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Aggregator;
 
 /**
  * Signals that an exception has occurred while retrieving {@link Aggregator}s.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorValues.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorValues.java
index f88adc7..a42ece2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorValues.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AggregatorValues.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
 
 import java.util.Collection;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipeline.java
index df596b2..45f7647 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipeline.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
 
 /**
  * A {@link DirectPipeline} is a {@link Pipeline} that returns
  * {@link DirectPipelineRunner.EvaluationResults} when it is
- * {@link com.google.cloud.dataflow.sdk.Pipeline#run()}.
+ * {@link org.apache.beam.sdk.Pipeline#run()}.
  */
 public class DirectPipeline extends Pipeline {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
index f4a5600..7dd0fdd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 
 import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
 import com.google.common.collect.ImmutableList;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
index 57e6116..198d04e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
@@ -15,63 +15,65 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
 
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindows;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunner;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunners;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+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.io.AvroIO;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+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.Partition;
+import org.apache.beam.sdk.transforms.Partition.PartitionFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.AssignWindows;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+import org.apache.beam.sdk.util.PerKeyCombineFnRunner;
+import org.apache.beam.sdk.util.PerKeyCombineFnRunners;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TypedPValue;
+
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
@@ -462,10 +464,10 @@ public class DirectPipelineRunner
 
   /**
    * The implementation may split the {@link KeyedCombineFn} into ADD, MERGE and EXTRACT phases (
-   * see {@code com.google.cloud.dataflow.sdk.runners.worker.CombineValuesFn}). In order to emulate
+   * see {@code org.apache.beam.sdk.runners.worker.CombineValuesFn}). In order to emulate
    * this for the {@link DirectPipelineRunner} and provide an experience closer to the service, go
    * through heavy serializability checks for the equivalent of the results of the ADD phase, but
-   * after the {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} shuffle, and the MERGE
+   * after the {@link org.apache.beam.sdk.transforms.GroupByKey} shuffle, and the MERGE
    * phase. Doing these checks ensure that not only is the accumulator coder serializable, but
    * the accumulator coder can actually serialize the data in question.
    */
@@ -629,7 +631,7 @@ public class DirectPipelineRunner
 
     /**
      * Retrieves the values indicated by the given {@link PCollectionView}.
-     * Note that within the {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context}
+     * Note that within the {@link org.apache.beam.sdk.transforms.DoFn.Context}
      * implementation a {@link PCollectionView} should convert from this representation to a
      * suitable side input value.
      */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7f25183..941debb 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
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.common.base.Preconditions;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
index 1acd71f..949f5da 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import com.google.auto.service.AutoService;
+
 import java.util.ServiceLoader;
 
 /**
@@ -26,7 +27,7 @@ import java.util.ServiceLoader;
  * and a concrete implementation of this interface.
  *
  * <p>Note that automatic registration of any
- * {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} requires users
+ * {@link org.apache.beam.sdk.options.PipelineOptions} requires users
  * conform to the limit that each {@link PipelineRunner}'s
  * {@link Class#getSimpleName() simple name} must be unique.
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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
index ee9e8df..84df5fd 100644
--- 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
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
+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 com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor}
+ * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor}
  * that records the transformation tree.
  *
  * <p>Provided for internal unit tests.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 23687c4..6aeaf5f 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
 import com.google.common.base.Preconditions;
 
 import java.util.Deque;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 60d308f..a6efc51 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
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+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.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
 import com.google.common.base.Preconditions;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
index 6342725..fe9c165 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
 
 /**
  * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
index 49576e5..e19ffe4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
index f034e2f..ef5581d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.io.IOException;
 import java.util.Queue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
index 504d68e..5479b00 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * A factory that creates {@link UncommittedBundle UncommittedBundles}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
index 0989fb5..12427d9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
index 40a062f..7a51251 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
index b581616..90c488e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
 
 /**
  * A callback for completing a bundle of input.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
index 7348de3..f6ea4af 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
 
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
index fd82cf2..d198903 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
+package org.apache.beam.sdk.runners.inprocess;
 
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
 
 /**
  * A {@link TransformEvaluator} that ignores all input and produces no output. The result of



[41/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
new file mode 100644
index 0000000..259ce08
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractCountryInfoFn;
+import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractEventDataFn;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/** Unit tests for {@link JoinExamples}. */
+@RunWith(JUnit4.class)
+public class JoinExamplesTest {
+
+  private static final TableRow row1 = new TableRow()
+        .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+        .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
+  private static final TableRow row2 = new TableRow()
+        .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+        .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
+  private static final TableRow row3 = new TableRow()
+        .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
+        .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
+  static final TableRow[] EVENTS = new TableRow[] {
+    row1, row2, row3
+  };
+  static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
+
+  private static final KV<String, String> kv1 = KV.of("VM",
+      "Date: 20141212, Actor1: LAOS, url: http://www.chicagotribune.com");
+  private static final KV<String, String> kv2 = KV.of("BE",
+      "Date: 20141213, Actor1: AFGHANISTAN, url: http://cnn.com");
+  private static final KV<String, String> kv3 = KV.of("BE", "Belgium");
+  private static final KV<String, String> kv4 = KV.of("VM", "Vietnam");
+
+  private static final TableRow cc1 = new TableRow()
+        .set("FIPSCC", "VM").set("HumanName", "Vietnam");
+  private static final TableRow cc2 = new TableRow()
+        .set("FIPSCC", "BE").set("HumanName", "Belgium");
+  static final TableRow[] CCS = new TableRow[] {
+    cc1, cc2
+  };
+  static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
+
+  static final String[] JOINED_EVENTS = new String[] {
+      "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
+          + "url: http://www.chicagotribune.com",
+      "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
+          + "url: http://cnn.com",
+      "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
+          + "url: http://cnn.com"
+    };
+
+  @Test
+  public void testExtractEventDataFn() {
+    DoFnTester<TableRow, KV<String, String>> extractEventDataFn =
+        DoFnTester.of(new ExtractEventDataFn());
+    List<KV<String, String>> results = extractEventDataFn.processBatch(EVENTS);
+    Assert.assertThat(results, CoreMatchers.hasItem(kv1));
+    Assert.assertThat(results, CoreMatchers.hasItem(kv2));
+  }
+
+  @Test
+  public void testExtractCountryInfoFn() {
+    DoFnTester<TableRow, KV<String, String>> extractCountryInfoFn =
+        DoFnTester.of(new ExtractCountryInfoFn());
+    List<KV<String, String>> results = extractCountryInfoFn.processBatch(CCS);
+    Assert.assertThat(results, CoreMatchers.hasItem(kv3));
+    Assert.assertThat(results, CoreMatchers.hasItem(kv4));
+  }
+
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testJoin() throws java.lang.Exception {
+    Pipeline p = TestPipeline.create();
+    PCollection<TableRow> input1 = p.apply("CreateEvent", Create.of(EVENT_ARRAY));
+    PCollection<TableRow> input2 = p.apply("CreateCC", Create.of(CC_ARRAY));
+
+    PCollection<String> output = JoinExamples.joinEvents(input1, input2);
+    PAssert.that(output).containsInAnyOrder(JOINED_EVENTS);
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
new file mode 100644
index 0000000..f3ffcea
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.ExtractTempFn;
+import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.List;
+
+/** Unit tests for {@link MaxPerKeyExamples}. */
+@RunWith(JUnit4.class)
+public class MaxPerKeyExamplesTest {
+
+  private static final TableRow row1 = new TableRow()
+        .set("month", "6").set("day", "21")
+        .set("year", "2014").set("mean_temp", "85.3")
+        .set("tornado", true);
+  private static final TableRow row2 = new TableRow()
+        .set("month", "7").set("day", "20")
+        .set("year", "2014").set("mean_temp", "75.4")
+        .set("tornado", false);
+  private static final TableRow row3 = new TableRow()
+        .set("month", "6").set("day", "18")
+        .set("year", "2014").set("mean_temp", "45.3")
+        .set("tornado", true);
+  private static final List<TableRow> TEST_ROWS = ImmutableList.of(row1, row2, row3);
+
+  private static final KV<Integer, Double> kv1 = KV.of(6, 85.3);
+  private static final KV<Integer, Double> kv2 = KV.of(6, 45.3);
+  private static final KV<Integer, Double> kv3 = KV.of(7, 75.4);
+
+  private static final List<KV<Integer, Double>> TEST_KVS = ImmutableList.of(kv1, kv2, kv3);
+
+  private static final TableRow resultRow1 = new TableRow()
+      .set("month", 6)
+      .set("max_mean_temp", 85.3);
+  private static final TableRow resultRow2 = new TableRow()
+      .set("month", 7)
+      .set("max_mean_temp", 75.4);
+
+
+  @Test
+  public void testExtractTempFn() {
+    DoFnTester<TableRow, KV<Integer, Double>> extractTempFn =
+        DoFnTester.of(new ExtractTempFn());
+    List<KV<Integer, Double>> results = extractTempFn.processBatch(TEST_ROWS);
+    Assert.assertThat(results, CoreMatchers.hasItem(kv1));
+    Assert.assertThat(results, CoreMatchers.hasItem(kv2));
+    Assert.assertThat(results, CoreMatchers.hasItem(kv3));
+  }
+
+  @Test
+  public void testFormatMaxesFn() {
+    DoFnTester<KV<Integer, Double>, TableRow> formatMaxesFnFn =
+        DoFnTester.of(new FormatMaxesFn());
+    List<TableRow> results = formatMaxesFnFn.processBatch(TEST_KVS);
+    Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
+    Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
new file mode 100644
index 0000000..048d803
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.TriggerExample.ExtractFlowInfo;
+import com.google.cloud.dataflow.examples.cookbook.TriggerExample.TotalFlow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Unit Tests for {@link TriggerExample}.
+ * The results generated by triggers are by definition non-deterministic and hence hard to test.
+ * The unit test does not test all aspects of the example.
+ */
+@RunWith(JUnit4.class)
+public class TriggerExampleTest {
+
+  private static final String[] INPUT =
+    {"01/01/2010 00:00:00,1108302,94,E,ML,36,100,29,0.0065,66,9,1,0.001,74.8,1,9,3,0.0028,71,1,9,"
+        + "12,0.0099,67.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,,,0", "01/01/2010 00:00:00,"
+            + "1100333,5,N,FR,9,0,39,,,9,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,"};
+
+  private static final List<TimestampedValue<String>> TIME_STAMPED_INPUT = Arrays.asList(
+      TimestampedValue.of("01/01/2010 00:00:00,1108302,5,W,ML,36,100,30,0.0065,66,9,1,0.001,"
+          + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,87.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,"
+          + ",,0", new Instant(60000)),
+      TimestampedValue.of("01/01/2010 00:00:00,1108302,110,E,ML,36,100,40,0.0065,66,9,1,0.001,"
+          + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,67.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,"
+          + ",,0", new Instant(1)),
+      TimestampedValue.of("01/01/2010 00:00:00,1108302,110,E,ML,36,100,50,0.0065,66,9,1,"
+          + "0.001,74.8,1,9,3,0.0028,71,1,9,12,0.0099,97.4,1,9,13,0.0121,50.0,1,,,,,0,,,,,0"
+          + ",,,,,0,,,,,0", new Instant(1)));
+
+  private static final TableRow OUT_ROW_1 = new TableRow()
+      .set("trigger_type", "default")
+      .set("freeway", "5").set("total_flow", 30)
+      .set("number_of_records", 1)
+      .set("isFirst", true).set("isLast", true)
+      .set("timing", "ON_TIME")
+      .set("window", "[1970-01-01T00:01:00.000Z..1970-01-01T00:02:00.000Z)");
+
+  private static final TableRow OUT_ROW_2 = new TableRow()
+      .set("trigger_type", "default")
+      .set("freeway", "110").set("total_flow", 90)
+      .set("number_of_records", 2)
+      .set("isFirst", true).set("isLast", true)
+      .set("timing", "ON_TIME")
+      .set("window", "[1970-01-01T00:00:00.000Z..1970-01-01T00:01:00.000Z)");
+
+  @Test
+  public void testExtractTotalFlow() {
+    DoFnTester<String, KV<String, Integer>> extractFlowInfow = DoFnTester
+        .of(new ExtractFlowInfo());
+
+    List<KV<String, Integer>> results = extractFlowInfow.processBatch(INPUT);
+    Assert.assertEquals(results.size(), 1);
+    Assert.assertEquals(results.get(0).getKey(), "94");
+    Assert.assertEquals(results.get(0).getValue(), new Integer(29));
+
+    List<KV<String, Integer>> output = extractFlowInfow.processBatch("");
+    Assert.assertEquals(output.size(), 0);
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testTotalFlow () {
+    Pipeline pipeline = TestPipeline.create();
+    PCollection<KV<String, Integer>> flow = pipeline
+        .apply(Create.timestamped(TIME_STAMPED_INPUT))
+        .apply(ParDo.of(new ExtractFlowInfo()));
+
+    PCollection<TableRow> totalFlow = flow
+        .apply(Window.<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(1))))
+        .apply(new TotalFlow("default"));
+
+    PCollection<TableRow> results =  totalFlow.apply(ParDo.of(new FormatResults()));
+
+
+    PAssert.that(results).containsInAnyOrder(OUT_ROW_1, OUT_ROW_2);
+    pipeline.run();
+
+  }
+
+  static class FormatResults extends DoFn<TableRow, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      TableRow element = c.element();
+      TableRow row = new TableRow()
+          .set("trigger_type", element.get("trigger_type"))
+          .set("freeway", element.get("freeway"))
+          .set("total_flow", element.get("total_flow"))
+          .set("number_of_records", element.get("number_of_records"))
+          .set("isFirst", element.get("isFirst"))
+          .set("isLast", element.get("isLast"))
+          .set("timing", element.get("timing"))
+          .set("window", element.get("window"));
+      c.output(row);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8.java
deleted file mode 100644
index 8705ed3..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8.java
+++ /dev/null
@@ -1,69 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import java.util.Arrays;
-
-/**
- * An example that counts words in Shakespeare, using Java 8 language features.
- *
- * <p>See {@link MinimalWordCount} for a comprehensive explanation.
- */
-public class MinimalWordCountJava8 {
-
-  public static void main(String[] args) {
-    DataflowPipelineOptions options = PipelineOptionsFactory.create()
-        .as(DataflowPipelineOptions.class);
-
-    options.setRunner(BlockingDataflowPipelineRunner.class);
-
-    // CHANGE 1 of 3: Your project ID is required in order to run your pipeline on the Google Cloud.
-    options.setProject("SET_YOUR_PROJECT_ID_HERE");
-
-    // CHANGE 2 of 3: Your Google Cloud Storage path is required for staging local files.
-    options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY");
-
-    Pipeline p = Pipeline.create(options);
-
-    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
-     .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
-         .withOutputType(new TypeDescriptor<String>() {}))
-     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
-     .apply(Count.<String>perElement())
-     .apply(MapElements
-         .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())
-         .withOutputType(new TypeDescriptor<String>() {}))
-
-     // CHANGE 3 of 3: The Google Cloud Storage path is required for outputting the results to.
-     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/GameStats.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/GameStats.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/GameStats.java
deleted file mode 100644
index 89832b2..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/GameStats.java
+++ /dev/null
@@ -1,340 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.Mean;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-
-/**
- * This class is the fourth in a series of four pipelines that tell a story in a 'gaming'
- * domain, following {@link UserScore}, {@link HourlyTeamScore}, and {@link LeaderBoard}.
- * New concepts: session windows and finding session duration; use of both
- * singleton and non-singleton side inputs.
- *
- * <p> This pipeline builds on the {@link LeaderBoard} functionality, and adds some "business
- * intelligence" analysis: abuse detection and usage patterns. The pipeline derives the Mean user
- * score sum for a window, and uses that information to identify likely spammers/robots. (The robots
- * have a higher click rate than the human users). The 'robot' users are then filtered out when
- * calculating the team scores.
- *
- * <p> Additionally, user sessions are tracked: that is, we find bursts of user activity using
- * session windows. Then, the mean session duration information is recorded in the context of
- * subsequent fixed windowing. (This could be used to tell us what games are giving us greater
- * user retention).
- *
- * <p> Run {@code com.google.cloud.dataflow.examples.complete.game.injector.Injector} to generate
- * pubsub data for this pipeline. The {@code Injector} documentation provides more detail.
- *
- * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
- * like this:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- *   --dataset=YOUR-DATASET
- *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
- * }
- * </pre>
- * where the BigQuery dataset you specify must already exist. The PubSub topic you specify should
- * be the same topic to which the Injector is publishing.
- */
-public class GameStats extends LeaderBoard {
-
-  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
-
-  private static DateTimeFormatter fmt =
-      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
-          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
-
-  /**
-   * Filter out all but those users with a high clickrate, which we will consider as 'spammy' uesrs.
-   * We do this by finding the mean total score per user, then using that information as a side
-   * input to filter out all but those user scores that are > (mean * SCORE_WEIGHT)
-   */
-  // [START DocInclude_AbuseDetect]
-  public static class CalculateSpammyUsers
-      extends PTransform<PCollection<KV<String, Integer>>, PCollection<KV<String, Integer>>> {
-    private static final Logger LOG = LoggerFactory.getLogger(CalculateSpammyUsers.class);
-    private static final double SCORE_WEIGHT = 2.5;
-
-    @Override
-    public PCollection<KV<String, Integer>> apply(PCollection<KV<String, Integer>> userScores) {
-
-      // Get the sum of scores for each user.
-      PCollection<KV<String, Integer>> sumScores = userScores
-          .apply("UserSum", Sum.<String>integersPerKey());
-
-      // Extract the score from each element, and use it to find the global mean.
-      final PCollectionView<Double> globalMeanScore = sumScores.apply(Values.<Integer>create())
-          .apply(Mean.<Integer>globally().asSingletonView());
-
-      // Filter the user sums using the global mean.
-      PCollection<KV<String, Integer>> filtered = sumScores
-          .apply(ParDo
-              .named("ProcessAndFilter")
-              // use the derived mean total score as a side input
-              .withSideInputs(globalMeanScore)
-              .of(new DoFn<KV<String, Integer>, KV<String, Integer>>() {
-                private final Aggregator<Long, Long> numSpammerUsers =
-                  createAggregator("SpammerUsers", new Sum.SumLongFn());
-                @Override
-                public void processElement(ProcessContext c) {
-                  Integer score = c.element().getValue();
-                  Double gmc = c.sideInput(globalMeanScore);
-                  if (score > (gmc * SCORE_WEIGHT)) {
-                    LOG.info("user " + c.element().getKey() + " spammer score " + score
-                        + " with mean " + gmc);
-                    numSpammerUsers.addValue(1L);
-                    c.output(c.element());
-                  }
-                }
-              }));
-      return filtered;
-    }
-  }
-  // [END DocInclude_AbuseDetect]
-
-  /**
-   * Calculate and output an element's session duration.
-   */
-  private static class UserSessionInfoFn extends DoFn<KV<String, Integer>, Integer>
-      implements RequiresWindowAccess {
-
-    @Override
-    public void processElement(ProcessContext c) {
-      IntervalWindow w = (IntervalWindow) c.window();
-      int duration = new Duration(
-          w.start(), w.end()).toPeriod().toStandardMinutes().getMinutes();
-      c.output(duration);
-    }
-  }
-
-
-  /**
-   * Options supported by {@link GameStats}.
-   */
-  static interface Options extends LeaderBoard.Options {
-    @Description("Numeric value of fixed window duration for user analysis, in minutes")
-    @Default.Integer(60)
-    Integer getFixedWindowDuration();
-    void setFixedWindowDuration(Integer value);
-
-    @Description("Numeric value of gap between user sessions, in minutes")
-    @Default.Integer(5)
-    Integer getSessionGap();
-    void setSessionGap(Integer value);
-
-    @Description("Numeric value of fixed window for finding mean of user session duration, "
-        + "in minutes")
-    @Default.Integer(30)
-    Integer getUserActivityWindowDuration();
-    void setUserActivityWindowDuration(Integer value);
-
-    @Description("Prefix used for the BigQuery table names")
-    @Default.String("game_stats")
-    String getTablePrefix();
-    void setTablePrefix(String value);
-  }
-
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is used to write information about team score sums.
-   */
-  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
-      configureWindowedWrite() {
-    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
-        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> { IntervalWindow w = (IntervalWindow) c.window();
-                 return fmt.print(w.start()); }));
-    tableConfigure.put("processing_time",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
-    return tableConfigure;
-  }
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is used to write information about mean user session time.
-   */
-  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<Double>>
-      configureSessionWindowWrite() {
-
-    Map<String, WriteWindowedToBigQuery.FieldInfo<Double>> tableConfigure =
-        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<Double>>();
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<Double>("STRING",
-          c -> { IntervalWindow w = (IntervalWindow) c.window();
-                 return fmt.print(w.start()); }));
-    tableConfigure.put("mean_duration",
-        new WriteWindowedToBigQuery.FieldInfo<Double>("FLOAT", c -> c.element()));
-    return tableConfigure;
-  }
-
-
-
-  public static void main(String[] args) throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    // Enforce that this pipeline is always run in streaming mode.
-    options.setStreaming(true);
-    // Allow the pipeline to be cancelled automatically.
-    options.setRunner(DataflowPipelineRunner.class);
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
-    Pipeline pipeline = Pipeline.create(options);
-
-    // Read Events from Pub/Sub using custom timestamps
-    PCollection<GameActionInfo> rawEvents = pipeline
-        .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic()))
-        .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()));
-
-    // Extract username/score pairs from the event stream
-    PCollection<KV<String, Integer>> userEvents =
-        rawEvents.apply("ExtractUserScore",
-          MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
-            .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
-
-    // 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())))
-          )
-
-      // Filter out everyone but those with (SCORE_WEIGHT * avg) clickrate.
-      // These might be robots/spammers.
-      .apply("CalculateSpammyUsers", new CalculateSpammyUsers())
-      // Derive a view from the collection of spammer users. It will be used as a side input
-      // in calculating the team score sums, below.
-      .apply("CreateSpammersView", View.<String, Integer>asMap());
-
-    // [START DocInclude_FilterAndCalc]
-    // Calculate the total score per team over fixed windows,
-    // and emit cumulative updates for late data. Uses the side input derived above-- the set of
-    // 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())))
-          )
-      // Filter out the detected spammer users, using the side input derived above.
-      .apply(ParDo.named("FilterOutSpammers")
-              .withSideInputs(spammersView)
-              .of(new DoFn<GameActionInfo, GameActionInfo>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  // If the user is not in the spammers Map, output the data element.
-                  if (c.sideInput(spammersView).get(c.element().getUser().trim()) == null) {
-                    c.output(c.element());
-                  }
-                }
-              }))
-      // Extract and sum teamname/score pairs from the event data.
-      .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
-      // [END DocInclude_FilterAndCalc]
-      // Write the result to BigQuery
-      .apply("WriteTeamSums",
-             new WriteWindowedToBigQuery<KV<String, Integer>>(
-                options.getTablePrefix() + "_team", configureWindowedWrite()));
-
-
-    // [START DocInclude_SessionCalc]
-    // Detect user sessions-- that is, a burst of activity separated by a gap from further
-    // activity. Find and record the mean session lengths.
-    // 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()))
-      // 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))
-      // Get the duration per session.
-      .apply("UserSessionActivity", ParDo.of(new UserSessionInfoFn()))
-      // [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()))))
-      // Find the mean session duration in each window.
-      .apply(Mean.<Integer>globally().withoutDefaults())
-      // Write this info to a BigQuery table.
-      .apply("WriteAvgSessionLength",
-             new WriteWindowedToBigQuery<Double>(
-                options.getTablePrefix() + "_sessions", configureSessionWindowWrite()));
-    // [END DocInclude_Rewindow]
-
-
-    // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
-    // command line.
-    PipelineResult result = pipeline.run();
-    dataflowUtils.waitToFinish(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java
deleted file mode 100644
index 58944c5..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java
+++ /dev/null
@@ -1,194 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.WithTimestamps;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-
-/**
- * This class is the second in a series of four pipelines that tell a story in a 'gaming'
- * domain, following {@link UserScore}. In addition to the concepts introduced in {@link UserScore},
- * new concepts include: windowing and element timestamps; use of {@code Filter.byPredicate()}.
- *
- * <p> This pipeline processes data collected from gaming events in batch, building on {@link
- * UserScore} but using fixed windows. It calculates the sum of scores per team, for each window,
- * optionally allowing specification of two timestamps before and after which data is filtered out.
- * This allows a model where late data collected after the intended analysis window can be included,
- * and any late-arriving data prior to the beginning of the analysis window can be removed as well.
- * By using windowing and adding element timestamps, we can do finer-grained analysis than with the
- * {@link UserScore} pipeline. However, our batch processing is high-latency, in that we don't get
- * results from plays at the beginning of the batch's time period until the batch is processed.
- *
- * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
- * like this:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- *   --dataset=YOUR-DATASET
- * }
- * </pre>
- * where the BigQuery dataset you specify must already exist.
- *
- * <p> Optionally include {@code --input} to specify the batch input file path.
- * To indicate a time after which the data should be filtered out, include the
- * {@code --stopMin} arg. E.g., {@code --stopMin=2015-10-18-23-59} indicates that any data
- * timestamped after 23:59 PST on 2015-10-18 should not be included in the analysis.
- * To indicate a time before which data should be filtered out, include the {@code --startMin} arg.
- * If you're using the default input specified in {@link UserScore},
- * "gs://dataflow-samples/game/gaming_data*.csv", then
- * {@code --startMin=2015-11-16-16-10 --stopMin=2015-11-17-16-10} are good values.
- */
-public class HourlyTeamScore extends UserScore {
-
-  private static DateTimeFormatter fmt =
-      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
-          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
-  private static DateTimeFormatter minFmt =
-      DateTimeFormat.forPattern("yyyy-MM-dd-HH-mm")
-          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
-
-
-  /**
-   * Options supported by {@link HourlyTeamScore}.
-   */
-  static interface Options extends UserScore.Options {
-
-    @Description("Numeric value of fixed window duration, in minutes")
-    @Default.Integer(60)
-    Integer getWindowDuration();
-    void setWindowDuration(Integer value);
-
-    @Description("String representation of the first minute after which to generate results,"
-        + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST."
-        + "Any input data timestamped prior to that minute won't be included in the sums.")
-    @Default.String("1970-01-01-00-00")
-    String getStartMin();
-    void setStartMin(String value);
-
-    @Description("String representation of the first minute for which to not generate results,"
-        + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST."
-        + "Any input data timestamped after that minute won't be included in the sums.")
-    @Default.String("2100-01-01-00-00")
-    String getStopMin();
-    void setStopMin(String value);
-
-    @Description("The BigQuery table name. Should not already exist.")
-    @Default.String("hourly_team_score")
-    String getTableName();
-    void setTableName(String value);
-  }
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is passed to the {@link WriteWindowedToBigQuery} constructor to write team score sums and
-   * includes information about window start time.
-   */
-  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
-      configureWindowedTableWrite() {
-    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfig =
-        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfig.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfig.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfig.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> { IntervalWindow w = (IntervalWindow) c.window();
-                 return fmt.print(w.start()); }));
-    return tableConfig;
-  }
-
-
-  /**
-   * Run a batch pipeline to do windowed analysis of the data.
-   */
-  // [START DocInclude_HTSMain]
-  public static void main(String[] args) throws Exception {
-    // Begin constructing a pipeline configured by commandline flags.
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    final Instant stopMinTimestamp = new Instant(minFmt.parseMillis(options.getStopMin()));
-    final Instant startMinTimestamp = new Instant(minFmt.parseMillis(options.getStartMin()));
-
-    // Read 'gaming' events from a text file.
-    pipeline.apply(TextIO.Read.from(options.getInput()))
-      // Parse the incoming data.
-      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
-
-      // Filter out data before and after the given times so that it is not included
-      // in the calculations. As we collect data in batches (say, by day), the batch for the day
-      // that we want to analyze could potentially include some late-arriving data from the previous
-      // day. If so, we want to weed it out. Similarly, if we include data from the following day
-      // (to scoop up late-arriving events from the day we're analyzing), we need to weed out events
-      // that fall after the time period we want to analyze.
-      // [START DocInclude_HTSFilters]
-      .apply("FilterStartTime", Filter.byPredicate(
-          (GameActionInfo gInfo)
-              -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
-      .apply("FilterEndTime", Filter.byPredicate(
-          (GameActionInfo gInfo)
-              -> gInfo.getTimestamp() < stopMinTimestamp.getMillis()))
-      // [END DocInclude_HTSFilters]
-
-      // [START DocInclude_HTSAddTsAndWindow]
-      // 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()))))
-      // [END DocInclude_HTSAddTsAndWindow]
-
-      // Extract and sum teamname/score pairs from the event data.
-      .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
-      .apply("WriteTeamScoreSums",
-        new WriteWindowedToBigQuery<KV<String, Integer>>(options.getTableName(),
-            configureWindowedTableWrite()));
-
-
-    pipeline.run();
-  }
-  // [END DocInclude_HTSMain]
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java
deleted file mode 100644
index cedd696..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java
+++ /dev/null
@@ -1,238 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-
-/**
- * This class is the third in a series of four pipelines that tell a story in a 'gaming' domain,
- * following {@link UserScore} and {@link HourlyTeamScore}. Concepts include: processing unbounded
- * data using fixed windows; use of custom timestamps and event-time processing; generation of
- * early/speculative results; using .accumulatingFiredPanes() to do cumulative processing of late-
- * arriving data.
- *
- * <p> This pipeline processes an unbounded stream of 'game events'. The calculation of the team
- * scores uses fixed windowing based on event time (the time of the game play event), not
- * processing time (the time that an event is processed by the pipeline). The pipeline calculates
- * the sum of scores per team, for each window. By default, the team scores are calculated using
- * one-hour windows.
- *
- * <p> In contrast-- to demo another windowing option-- the user scores are calculated using a
- * global window, which periodically (every ten minutes) emits cumulative user score sums.
- *
- * <p> In contrast to the previous pipelines in the series, which used static, finite input data,
- * here we're using an unbounded data source, which lets us provide speculative results, and allows
- * handling of late data, at much lower latency. We can use the early/speculative results to keep a
- * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct
- * results, e.g. for 'team prizes'. We're now outputing window results as they're
- * calculated, giving us much lower latency than with the previous batch examples.
- *
- * <p> Run {@link injector.Injector} to generate pubsub data for this pipeline.  The Injector
- * documentation provides more detail on how to do this.
- *
- * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
- * like this:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- *   --dataset=YOUR-DATASET
- *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
- * }
- * </pre>
- * where the BigQuery dataset you specify must already exist.
- * The PubSub topic you specify should be the same topic to which the Injector is publishing.
- */
-public class LeaderBoard extends HourlyTeamScore {
-
-  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
-
-  private static DateTimeFormatter fmt =
-      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
-          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
-  static final Duration FIVE_MINUTES = Duration.standardMinutes(5);
-  static final Duration TEN_MINUTES = Duration.standardMinutes(10);
-
-
-  /**
-   * Options supported by {@link LeaderBoard}.
-   */
-  static interface Options extends HourlyTeamScore.Options, DataflowExampleOptions {
-
-    @Description("Pub/Sub topic to read from")
-    @Validation.Required
-    String getTopic();
-    void setTopic(String value);
-
-    @Description("Numeric value of fixed window duration for team analysis, in minutes")
-    @Default.Integer(60)
-    Integer getTeamWindowDuration();
-    void setTeamWindowDuration(Integer value);
-
-    @Description("Numeric value of allowed data lateness, in minutes")
-    @Default.Integer(120)
-    Integer getAllowedLateness();
-    void setAllowedLateness(Integer value);
-
-    @Description("Prefix used for the BigQuery table names")
-    @Default.String("leaderboard")
-    String getTableName();
-    void setTableName(String value);
-  }
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is used to write team score sums and includes event timing information.
-   */
-  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
-      configureWindowedTableWrite() {
-
-    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
-        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> { IntervalWindow w = (IntervalWindow) c.window();
-                 return fmt.print(w.start()); }));
-    tableConfigure.put("processing_time",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
-    tableConfigure.put("timing",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> c.pane().getTiming().toString()));
-    return tableConfigure;
-  }
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is used to write user score sums.
-   */
-  protected static Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>
-      configureGlobalWindowBigQueryWrite() {
-
-    Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
-        configureBigQueryWrite();
-    tableConfigure.put("processing_time",
-        new WriteToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
-    return tableConfigure;
-  }
-
-
-  public static void main(String[] args) throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    // Enforce that this pipeline is always run in streaming mode.
-    options.setStreaming(true);
-    // For example purposes, allow the pipeline to be easily cancelled instead of running
-    // continuously.
-    options.setRunner(DataflowPipelineRunner.class);
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
-    Pipeline pipeline = Pipeline.create(options);
-
-    // Read game events from Pub/Sub using custom timestamps, which are extracted from the pubsub
-    // data elements, and parse the data.
-    PCollection<GameActionInfo> gameEvents = pipeline
-        .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic()))
-        .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()));
-
-    // [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())))
-          // We will get early (speculative) results as well as cumulative
-          // processing of late data.
-          .triggering(
-            AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(FIVE_MINUTES))
-            .withLateFirings(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(TEN_MINUTES)))
-          .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))
-          .accumulatingFiredPanes())
-        // Extract and sum teamname/score pairs from the event data.
-        .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
-        // Write the results to BigQuery.
-        .apply("WriteTeamScoreSums",
-               new WriteWindowedToBigQuery<KV<String, Integer>>(
-                  options.getTableName() + "_team", configureWindowedTableWrite()));
-    // [END DocInclude_WindowAndTrigger]
-
-    // [START DocInclude_ProcTimeTrigger]
-    // 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())
-          // Get periodic results every ten minutes.
-              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(TEN_MINUTES)))
-              .accumulatingFiredPanes()
-              .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())))
-        // Extract and sum username/score pairs from the event data.
-        .apply("ExtractUserScore", new ExtractAndSumScore("user"))
-        // Write the results to BigQuery.
-        .apply("WriteUserScoreSums",
-               new WriteToBigQuery<KV<String, Integer>>(
-                  options.getTableName() + "_user", configureGlobalWindowBigQueryWrite()));
-    // [END DocInclude_ProcTimeTrigger]
-
-    // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
-    // command line.
-    PipelineResult result = pipeline.run();
-    dataflowUtils.waitToFinish(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/README.md
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/README.md b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/README.md
deleted file mode 100644
index 79b55ce..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/README.md
+++ /dev/null
@@ -1,113 +0,0 @@
-
-# 'Gaming' examples
-
-
-This directory holds a series of example Dataflow pipelines in a simple 'mobile
-gaming' domain. They all require Java 8.  Each pipeline successively introduces
-new concepts, and gives some examples of using Java 8 syntax in constructing
-Dataflow pipelines. Other than usage of Java 8 lambda expressions, the concepts
-that are used apply equally well in Java 7.
-
-In the gaming scenario, many users play, as members of different teams, over
-the course of a day, and their actions are logged for processing. Some of the
-logged game events may be late-arriving, if users play on mobile devices and go
-transiently offline for a period.
-
-The scenario includes not only "regular" users, but "robot users", which have a
-higher click rate than the regular users, and may move from team to team.
-
-The first two pipelines in the series use pre-generated batch data samples. The
-second two pipelines read from a [PubSub](https://cloud.google.com/pubsub/)
-topic input.  For these examples, you will also need to run the
-`injector.Injector` program, which generates and publishes the gaming data to
-PubSub. The javadocs for each pipeline have more detailed information on how to
-run that pipeline.
-
-All of these pipelines write their results to BigQuery table(s).
-
-
-## The pipelines in the 'gaming' series
-
-### UserScore
-
-The first pipeline in the series is `UserScore`. This pipeline does batch
-processing of data collected from gaming events. It calculates the sum of
-scores per user, over an entire batch of gaming data (collected, say, for each
-day). The batch processing will not include any late data that arrives after
-the day's cutoff point.
-
-### HourlyTeamScore
-
-The next pipeline in the series is `HourlyTeamScore`. This pipeline also
-processes data collected from gaming events in batch. It builds on `UserScore`,
-but uses [fixed windows](https://cloud.google.com/dataflow/model/windowing), by
-default an hour in duration. It calculates the sum of scores per team, for each
-window, optionally allowing specification of two timestamps before and after
-which data is filtered out. This allows a model where late data collected after
-the intended analysis window can be included in the analysis, and any late-
-arriving data prior to the beginning of the analysis window can be removed as
-well.
-
-By using windowing and adding element timestamps, we can do finer-grained
-analysis than with the `UserScore` pipeline — we're now tracking scores for
-each hour rather than over the course of a whole day. However, our batch
-processing is high-latency, in that we don't get results from plays at the
-beginning of the batch's time period until the complete batch is processed.
-
-### LeaderBoard
-
-The third pipeline in the series is `LeaderBoard`. This pipeline processes an
-unbounded stream of 'game events' from a PubSub topic. The calculation of the
-team scores uses fixed windowing based on event time (the time of the game play
-event), not processing time (the time that an event is processed by the
-pipeline). The pipeline calculates the sum of scores per team, for each window.
-By default, the team scores are calculated using one-hour windows.
-
-In contrast — to demo another windowing option — the user scores are calculated
-using a global window, which periodically (every ten minutes) emits cumulative
-user score sums.
-
-In contrast to the previous pipelines in the series, which used static, finite
-input data, here we're using an unbounded data source, which lets us provide
-_speculative_ results, and allows handling of late data, at much lower latency.
-E.g., we could use the early/speculative results to keep a 'leaderboard'
-updated in near-realtime. Our handling of late data lets us generate correct
-results, e.g. for 'team prizes'. We're now outputing window results as they're
-calculated, giving us much lower latency than with the previous batch examples.
-
-### GameStats
-
-The fourth pipeline in the series is `GameStats`. This pipeline builds
-on the `LeaderBoard` functionality — supporting output of speculative and late
-data — and adds some "business intelligence" analysis: identifying abuse
-detection. The pipeline derives the Mean user score sum for a window, and uses
-that information to identify likely spammers/robots. (The injector is designed
-so that the "robots" have a higher click rate than the "real" users). The robot
-users are then filtered out when calculating the team scores.
-
-Additionally, user sessions are tracked: that is, we find bursts of user
-activity using session windows. Then, the mean session duration information is
-recorded in the context of subsequent fixed windowing. (This could be used to
-tell us what games are giving us greater user retention).
-
-### Running the PubSub Injector
-
-The `LeaderBoard` and `GameStats` example pipelines read unbounded data
-from a PubSub topic.
-
-Use the `injector.Injector` program to generate this data and publish to a
-PubSub topic. See the `Injector`javadocs for more information on how to run the
-injector. Set up the injector before you start one of these pipelines. Then,
-when you start the pipeline, pass as an argument the name of that PubSub topic.
-See the pipeline javadocs for the details.
-
-## Viewing the results in BigQuery
-
-All of the pipelines write their results to BigQuery.  `UserScore` and
-`HourlyTeamScore` each write one table, and `LeaderBoard` and
-`GameStats` each write two. The pipelines have default table names that
-you can override when you start up the pipeline if those tables already exist.
-
-Depending on the windowing intervals defined in a given pipeline, you may have
-to wait for a while (more than an hour) before you start to see results written
-to the BigQuery tables.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/UserScore.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/UserScore.java
deleted file mode 100644
index 7babc6e..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/UserScore.java
+++ /dev/null
@@ -1,240 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.apache.avro.reflect.Nullable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * This class is the first in a series of four pipelines that tell a story in a 'gaming' domain.
- * Concepts: batch processing; reading input from Google Cloud Storage and writing output to
- * BigQuery; using standalone DoFns; use of the sum by key transform; examples of
- * Java 8 lambda syntax.
- *
- * <p> In this gaming scenario, many users play, as members of different teams, over the course of a
- * day, and their actions are logged for processing.  Some of the logged game events may be late-
- * arriving, if users play on mobile devices and go transiently offline for a period.
- *
- * <p> This pipeline does batch processing of data collected from gaming events. It calculates the
- * sum of scores per user, over an entire batch of gaming data (collected, say, for each day). The
- * batch processing will not include any late data that arrives after the day's cutoff point.
- *
- * <p> To execute this pipeline using the Dataflow service and static example input data, specify
- * the pipeline configuration like this:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- *   --dataset=YOUR-DATASET
- * }
- * </pre>
- * where the BigQuery dataset you specify must already exist.
- *
- * <p> Optionally include the --input argument to specify a batch input file.
- * See the --input default value for example batch data file, or use {@link injector.Injector} to
- * generate your own batch data.
-  */
-public class UserScore {
-
-  /**
-   * Class to hold info about a game event.
-   */
-  @DefaultCoder(AvroCoder.class)
-  static class GameActionInfo {
-    @Nullable String user;
-    @Nullable String team;
-    @Nullable Integer score;
-    @Nullable Long timestamp;
-
-    public GameActionInfo() {}
-
-    public GameActionInfo(String user, String team, Integer score, Long timestamp) {
-      this.user = user;
-      this.team = team;
-      this.score = score;
-      this.timestamp = timestamp;
-    }
-
-    public String getUser() {
-      return this.user;
-    }
-    public String getTeam() {
-      return this.team;
-    }
-    public Integer getScore() {
-      return this.score;
-    }
-    public String getKey(String keyname) {
-      if (keyname.equals("team")) {
-        return this.team;
-      } else {  // return username as default
-        return this.user;
-      }
-    }
-    public Long getTimestamp() {
-      return this.timestamp;
-    }
-  }
-
-
-  /**
-   * Parses the raw game event info into GameActionInfo objects. Each event line has the following
-   * format: username,teamname,score,timestamp_in_ms,readable_time
-   * e.g.:
-   * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224
-   * The human-readable time string is not used here.
-   */
-  static class ParseEventFn extends DoFn<String, GameActionInfo> {
-
-    // Log and count parse errors.
-    private static final Logger LOG = LoggerFactory.getLogger(ParseEventFn.class);
-    private final Aggregator<Long, Long> numParseErrors =
-        createAggregator("ParseErrors", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] components = c.element().split(",");
-      try {
-        String user = components[0].trim();
-        String team = components[1].trim();
-        Integer score = Integer.parseInt(components[2].trim());
-        Long timestamp = Long.parseLong(components[3].trim());
-        GameActionInfo gInfo = new GameActionInfo(user, team, score, timestamp);
-        c.output(gInfo);
-      } catch (ArrayIndexOutOfBoundsException | NumberFormatException e) {
-        numParseErrors.addValue(1L);
-        LOG.info("Parse error on " + c.element() + ", " + e.getMessage());
-      }
-    }
-  }
-
-  /**
-   * A transform to extract key/score information from GameActionInfo, and sum the scores. The
-   * constructor arg determines whether 'team' or 'user' info is extracted.
-   */
-  // [START DocInclude_USExtractXform]
-  public static class ExtractAndSumScore
-      extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> {
-
-    private final String field;
-
-    ExtractAndSumScore(String field) {
-      this.field = field;
-    }
-
-    @Override
-    public PCollection<KV<String, Integer>> apply(
-        PCollection<GameActionInfo> gameInfo) {
-
-      return gameInfo
-        .apply(MapElements
-            .via((GameActionInfo gInfo) -> KV.of(gInfo.getKey(field), gInfo.getScore()))
-            .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}))
-        .apply(Sum.<String>integersPerKey());
-    }
-  }
-  // [END DocInclude_USExtractXform]
-
-
-  /**
-   * Options supported by {@link UserScore}.
-   */
-  public static interface Options extends PipelineOptions {
-
-    @Description("Path to the data file(s) containing game data.")
-    // The default maps to two large Google Cloud Storage files (each ~12GB) holding two subsequent
-    // day's worth (roughly) of data.
-    @Default.String("gs://dataflow-samples/game/gaming_data*.csv")
-    String getInput();
-    void setInput(String value);
-
-    @Description("BigQuery Dataset to write tables to. Must already exist.")
-    @Validation.Required
-    String getDataset();
-    void setDataset(String value);
-
-    @Description("The BigQuery table name. Should not already exist.")
-    @Default.String("user_score")
-    String getTableName();
-    void setTableName(String value);
-  }
-
-  /**
-   * Create a map of information that describes how to write pipeline output to BigQuery. This map
-   * is passed to the {@link WriteToBigQuery} constructor to write user score sums.
-   */
-  protected static Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>
-    configureBigQueryWrite() {
-    Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
-        new HashMap<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("user",
-        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("STRING", c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER", c -> c.element().getValue()));
-    return tableConfigure;
-  }
-
-
-  /**
-   * Run a batch pipeline.
-   */
- // [START DocInclude_USMain]
-  public static void main(String[] args) throws Exception {
-    // Begin constructing a pipeline configured by commandline flags.
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline pipeline = Pipeline.create(options);
-
-    // Read events from a text file and parse them.
-    pipeline.apply(TextIO.Read.from(options.getInput()))
-      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
-      // Extract and sum username/score pairs from the event data.
-      .apply("ExtractUserScore", new ExtractAndSumScore("user"))
-      .apply("WriteUserScoreSums",
-          new WriteToBigQuery<KV<String, Integer>>(options.getTableName(),
-                                                   configureBigQueryWrite()));
-
-    // Run the batch pipeline.
-    pipeline.run();
-  }
-  // [END DocInclude_USMain]
-
-}


[68/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
index 77a8de6..44f4ecb 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
@@ -17,20 +17,32 @@
  */
 package org.apache.beam.runners.flink.streaming;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointReader;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointUtils;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointWriter;
-import com.google.cloud.dataflow.sdk.coders.*;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.state.*;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.DelegateCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.transforms.CombineWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
@@ -38,10 +50,11 @@ import org.apache.flink.runtime.util.DataInputDeserializer;
 import org.joda.time.Instant;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import static org.junit.Assert.assertEquals;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
 
 public class StateSerializationTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
index 83c1661..1efb42f 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
@@ -18,18 +18,20 @@
 package org.apache.beam.runners.flink.streaming;
 
 import org.apache.beam.runners.flink.FlinkTestPipeline;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.Joiner;
+
 import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceITCase.java
index f36028e..8a5de15 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceITCase.java
@@ -17,38 +17,36 @@
  */
 package org.apache.beam.runners.flink.streaming;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.runners.flink.FlinkTestPipeline;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
-import org.apache.beam.runners.flink.FlinkTestPipeline;
+
 import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.joda.time.Instant;
-import org.junit.internal.ArrayComparisonFailure;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.List;
 import java.util.NoSuchElementException;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
+import javax.annotation.Nullable;
 
 
 public class UnboundedSourceITCase extends StreamingProgramTestBase {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
index e850dd6..e6b7f64 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
@@ -17,25 +17,26 @@
  */
 package org.apache.beam.runners.flink.util;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+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.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 
 /**
- * Copied from {@link com.google.cloud.dataflow.examples.JoinExamples} because the code
+ * Copied from {@link org.apache.beam.examples.JoinExamples} because the code
  * is private there.
  */
 public class JoinExamples {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c04a9fc..7893975 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -177,8 +177,8 @@
           <windowtitle>Google Cloud Dataflow Java Runner ${project.version}</windowtitle>
           <doctitle>Google Cloud Dataflow Runner for Java, version ${project.version}</doctitle>
 
-          <subpackages>com.google.cloud.dataflow.sdk</subpackages>
-          <additionalparam>-exclude com.google.cloud.dataflow.sdk.runners.worker:com.google.cloud.dataflow.sdk.runners.dataflow:com.google.cloud.dataflow.sdk.util:com.google.cloud.dataflow.sdk.runners.inprocess ${dataflow.javadoc_opts}</additionalparam>
+          <subpackages>org.apache.beam.sdk</subpackages>
+          <additionalparam>-exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess ${dataflow.javadoc_opts}</additionalparam>
           <use>false</use>
           <quiet>true</quiet>
           <bottom><![CDATA[<br>]]></bottom>
@@ -272,19 +272,19 @@
               </filters>
               <relocations>
                 <!-- TODO: Once ready, change the following pattern to 'com'
-                     only, exclude 'com.google.cloud.dataflow.**', and remove
+                     only, exclude 'org.apache.beam.**', and remove
                      the second relocation. -->
                 <relocation>
                   <pattern>com.google.common</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.common</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.common</shadedPattern>
                 </relocation>
                 <relocation>
                   <pattern>com.google.thirdparty</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.thirdparty</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.thirdparty</shadedPattern>
                 </relocation>
                 <relocation>
                   <pattern>com.google.cloud.bigtable</pattern>
-                  <shadedPattern>com.google.cloud.dataflow.sdk.repackaged.com.google.cloud.bigtable</shadedPattern>
+                  <shadedPattern>org.apache.beam.sdk.repackaged.com.google.cloud.bigtable</shadedPattern>
                   <excludes>
                     <exclude>com.google.cloud.bigtable.config.BigtableOptions*</exclude>
                     <exclude>com.google.cloud.bigtable.config.CredentialOptions*</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
index 6bbafdd..5b52d30 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
+import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
index 3f0503e..0464fb4 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import org.apache.beam.sdk.annotations.Experimental;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
index 1be93eb..7e0ef48 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.util.DataflowPathValidator;
+import org.apache.beam.sdk.util.DataflowTransport;
+import org.apache.beam.sdk.util.GcsStager;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PathValidator;
+import org.apache.beam.sdk.util.Stager;
 
 import com.google.api.services.dataflow.Dataflow;
-import com.google.cloud.dataflow.sdk.util.DataflowPathValidator;
-import com.google.cloud.dataflow.sdk.util.DataflowTransport;
-import com.google.cloud.dataflow.sdk.util.GcsStager;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.PathValidator;
-import com.google.cloud.dataflow.sdk.util.Stager;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
index dbfafd1..4eae85a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
+
+import org.apache.beam.sdk.runners.DataflowPipeline;
 
-import com.google.cloud.dataflow.sdk.runners.DataflowPipeline;
 import com.google.common.base.MoreObjects;
 
 import org.joda.time.DateTimeUtils;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
index 44a9b00..6b44e7a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
index 3cd7b03..da51230 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import org.apache.beam.sdk.annotations.Experimental;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
index 7705b66..a28b756 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 /**
  * Options that are used exclusively within the Dataflow worker harness.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
index ebd42d9..dc840d8 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.options;
+package org.apache.beam.sdk.options;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
index 9096c2b..0f4126f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
@@ -15,19 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
 import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
+import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
index 4a4f100..728baa7 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 /**
  * An exception that is thrown if the unique job name constraint of the Dataflow

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
index 1f52c6a..4d2025f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 /**
  * An exception that is thrown if the existing job has already been updated within the Dataflow

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
index 495ca5a..cc002f1 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 /**
  * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
index a22d13c..ed66fb7 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import java.util.Objects;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
index 3dbb007..bc8a903 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
index 5792941..65c327f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 /**
  * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
index 7b97c7d..4d91a38 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipeline.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * A {@link DataflowPipeline} is a {@link Pipeline} that returns a
  * {@link DataflowPipelineJob} when it is
- * {@link com.google.cloud.dataflow.sdk.Pipeline#run()}.
+ * {@link org.apache.beam.sdk.Pipeline#run()}.
  *
  * <p>This is not intended for use by users of Cloud Dataflow.
  * Instead, use {@link Pipeline#create(PipelineOptions)} to initialize a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
index 632be6d..428a6ed 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
@@ -15,9 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
+
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import org.apache.beam.sdk.runners.dataflow.DataflowMetricUpdateExtractor;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+import org.apache.beam.sdk.util.MonitoringUtil;
 
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.util.BackOff;
@@ -29,14 +38,6 @@ import com.google.api.services.dataflow.model.Job;
 import com.google.api.services.dataflow.model.JobMessage;
 import com.google.api.services.dataflow.model.JobMetrics;
 import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowMetricUpdateExtractor;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
index 8aaa7cc..9333d7d 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 
 import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
 import com.google.common.collect.ImmutableList;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
index 2818251..18e2cc9 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
@@ -15,15 +15,106 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
+
+import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName;
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
 
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximatePTransformName;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
-import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInEmptyWindows;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
+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;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+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.FileBasedSink;
+import org.apache.beam.sdk.io.PubsubIO;
+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;
+import org.apache.beam.sdk.options.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineWorkerPoolOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.JobSpecification;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.runners.dataflow.AssignWindows;
+import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import org.apache.beam.sdk.runners.dataflow.PubsubIOTranslator;
+import org.apache.beam.sdk.runners.dataflow.ReadTranslator;
+import org.apache.beam.sdk.runners.worker.IsmFormat;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+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.View;
+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;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.DataflowReleaseInfo;
+import org.apache.beam.sdk.util.DataflowTransport;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.PathValidator;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Reshuffle;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.ValueWithRecordId;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+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.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.services.clouddebugger.v2.Clouddebugger;
@@ -35,96 +126,6 @@ import com.google.api.services.dataflow.model.DataflowPackage;
 import com.google.api.services.dataflow.model.Job;
 import com.google.api.services.dataflow.model.ListJobsResponse;
 import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.ShardNameTemplate;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.io.Write;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.JobSpecification;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.runners.dataflow.AssignWindows;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.util.DataflowTransport;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.PCollectionViews;
-import com.google.cloud.dataflow.sdk.util.PathValidator;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Reshuffle;
-import com.google.cloud.dataflow.sdk.util.SystemDoFnInternal;
-import com.google.cloud.dataflow.sdk.util.ValueWithRecordId;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
@@ -807,7 +808,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
    * Dataflow runner in batch mode.
    *
    * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
@@ -918,7 +919,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsIterable View.AsIterable} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
    * Dataflow runner in batch mode.
    *
    * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
@@ -951,7 +952,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsList View.AsList} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
    * Dataflow runner in batch mode.
    *
    * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
@@ -1107,7 +1108,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMap View.AsMap} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for the
    * Dataflow runner in batch mode.
    *
    * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
@@ -1134,7 +1135,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
    * {@code [1, size of map]}.
    *
    * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
    * a warning to users to specify a deterministic key coder.
    */
   static class BatchViewAsMap<K, V>
@@ -1281,7 +1282,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMultimap View.AsMultimap} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
    * Dataflow runner in batch mode.
    *
    * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
@@ -1308,7 +1309,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
    * {@code [1, size of map]}.
    *
    * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
    * a warning to users to specify a deterministic key coder.
    */
   static class BatchViewAsMultimap<K, V>
@@ -1945,7 +1946,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
     @Override
     public void verifyDeterministic()
-        throws com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException {
+        throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
       verifyDeterministic("Expected transform coder to be deterministic.", transformCoder);
       verifyDeterministic("Expected map coder to be deterministic.", originalMapCoder);
     }
@@ -2036,7 +2037,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation which overrides
-   * {@link com.google.cloud.dataflow.sdk.io.Write.Bound Write.Bound} to provide Google
+   * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound} to provide Google
    * Cloud Dataflow specific path validation of {@link FileBasedSink}s.
    */
   private static class BatchWrite<T> extends PTransform<PCollection<T>, PDone> {
@@ -2064,7 +2065,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation which overrides
-   * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write.Bound TextIO.Write.Bound} with
+   * {@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.
    */
@@ -2169,7 +2170,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation which overrides
-   * {@link com.google.cloud.dataflow.sdk.io.AvroIO.Write.Bound AvroIO.Write.Bound} with
+   * {@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.
    */
@@ -2271,7 +2272,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized (non-)implementation for
-   * {@link com.google.cloud.dataflow.sdk.io.Write.Bound Write.Bound}
+   * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound}
    * for the Dataflow runner in streaming mode.
    */
   private static class StreamingWrite<T> extends PTransform<PCollection<T>, PDone> {
@@ -2295,7 +2296,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Write PubsubIO.Write} for the
+   * {@link org.apache.beam.sdk.io.PubsubIO.Write PubsubIO.Write} for the
    * Dataflow runner in streaming mode.
    *
    * <p>For internal use only. Subject to change at any time.
@@ -2330,7 +2331,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded Read.Unbounded} for the
+   * {@link org.apache.beam.sdk.io.Read.Unbounded Read.Unbounded} for the
    * Dataflow runner in streaming mode.
    *
    * <p>In particular, if an UnboundedSource requires deduplication, then features of WindmillSink
@@ -2446,7 +2447,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.Create.Values Create.Values} for the
+   * {@link org.apache.beam.sdk.transforms.Create.Values Create.Values} for the
    * Dataflow runner in streaming mode.
    */
   private static class StreamingCreate<T> extends PTransform<PInput, PCollection<T>> {
@@ -2564,7 +2565,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsMap View.AsMap}
+   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap}
    * for the Dataflow runner in streaming mode.
    */
   private static class StreamingViewAsMap<K, V>
@@ -2606,7 +2607,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized expansion for {@link
-   * com.google.cloud.dataflow.sdk.transforms.View.AsMultimap View.AsMultimap} for the
+   * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
    * Dataflow runner in streaming mode.
    */
   private static class StreamingViewAsMultimap<K, V>
@@ -2651,7 +2652,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsList View.AsList} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
    * Dataflow runner in streaming mode.
    */
   private static class StreamingViewAsList<T>
@@ -2683,7 +2684,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized implementation for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsIterable View.AsIterable} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
    * Dataflow runner in streaming mode.
    */
   private static class StreamingViewAsIterable<T>
@@ -2722,7 +2723,7 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
 
   /**
    * Specialized expansion for
-   * {@link com.google.cloud.dataflow.sdk.transforms.View.AsSingleton View.AsSingleton} for the
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
    * Dataflow runner in streaming mode.
    */
   private static class StreamingViewAsSingleton<T>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
index 8f8c653..7ea44d7 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import org.apache.beam.sdk.annotations.Experimental;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
index fa7067e..0a71f65 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
@@ -15,21 +15,65 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray;
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.getString;
+package org.apache.beam.sdk.runners;
+
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+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 org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDictionary;
+import static org.apache.beam.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addLong;
+import static org.apache.beam.sdk.util.Structs.addObject;
+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 org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
+import org.apache.beam.sdk.runners.dataflow.BigQueryIOTranslator;
+import org.apache.beam.sdk.runners.dataflow.PubsubIOTranslator;
+import org.apache.beam.sdk.runners.dataflow.ReadTranslator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+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.View;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.DoFnInfo;
+import org.apache.beam.sdk.util.OutputReference;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypedPValue;
+
 import com.google.api.services.dataflow.model.AutoscalingSettings;
 import com.google.api.services.dataflow.model.DataflowPackage;
 import com.google.api.services.dataflow.model.Disk;
@@ -37,48 +81,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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
-import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.DoFnInfo;
-import com.google.cloud.dataflow.sdk.util.OutputReference;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
index 8b9f3f4..c1f0e74 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners;
+package org.apache.beam.sdk.runners;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
index b9d67ad..b068f50 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+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.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
index 203ce4f..51d7000 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
+
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.WindowedValue;
 
 import com.google.api.client.json.JsonFactory;
 import com.google.api.services.bigquery.model.TableReference;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
index 3473c41..a3f9fda 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
@@ -15,21 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
+
+import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addStringList;
 
 import static com.google.api.client.util.Base64.encodeBase64String;
-import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
 import static com.google.common.base.Preconditions.checkArgument;
 
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.CloudObject;
+
 import com.google.api.services.dataflow.model.SourceMetadata;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
index 5c1dd95..088ce63 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 import com.google.common.collect.HashMultimap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
index 5fc1eb7..35ef229 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import com.google.api.services.dataflow.model.MetricStructuredName;
 import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
 
 import java.util.Collection;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
index b54d5c6..e299bec 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
 
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.WindowedValue;
 
 /**
  * Pubsub transform support code for the Dataflow backend.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
index 4998af3..0c394a1 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
@@ -15,22 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDictionary;
+import static org.apache.beam.sdk.util.Structs.addLong;
+
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.PValue;
 
 import com.google.api.services.dataflow.model.SourceMetadata;
-import com.google.cloud.dataflow.sdk.io.FileBasedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.values.PValue;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
index d1484ce..3630ea7 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Implementation of the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
+ * Implementation of the {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
  */
-package com.google.cloud.dataflow.sdk.runners.dataflow;
+package org.apache.beam.sdk.runners.dataflow;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
index 2f920a6..171231c 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
 
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
 
 /**
  * A set of options used to configure the {@link TestPipeline}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
index e961066..ef3333d 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
@@ -15,23 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.testing;
+package org.apache.beam.sdk.testing;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.DataflowJobExecutionException;
+import org.apache.beam.sdk.runners.DataflowPipelineJob;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.MonitoringUtil.JobMessagesHandler;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 import com.google.api.services.dataflow.model.JobMessage;
 import com.google.api.services.dataflow.model.JobMetrics;
 import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowJobExecutionException;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
 import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
index aeb864a..d0388a3 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
 import com.google.common.base.Preconditions;
 
 import java.io.IOException;



[60/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7c86d04..e97d3bd 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
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.util.ExecutableTrigger;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.state.MergingStateAccessor;
-import com.google.cloud.dataflow.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+
 import com.google.common.base.Joiner;
 
 import org.joda.time.Instant;
@@ -39,7 +40,7 @@ import javax.annotation.Nullable;
  * {@link WindowFn}, and then passed to the associated {@code Trigger} to determine if the
  * {@code Window}s contents should be output.
  *
- * <p>See {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} and {@link Window}
+ * <p>See {@link org.apache.beam.sdk.transforms.GroupByKey} and {@link Window}
  * for more information about how grouping with windows works.
  *
  * <p>The elements that are assigned to a window since the last time it was fired (or since the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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
index 017fd87..e8bd52b 100644
--- 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 /**
  * Anything that can be used to create an instance of a {@code Trigger} implements this interface.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 e643339..2c162f2 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
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+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.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 
@@ -38,11 +38,11 @@ import javax.annotation.Nullable;
  * {@link PCollection} into finite windows according to a {@link WindowFn}.
  * The output of {@code Window} contains the same elements as input, but they
  * have been logically assigned to windows. The next
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey GroupByKeys},
+ * {@link org.apache.beam.sdk.transforms.GroupByKey GroupByKeys},
  * including one within composite transforms, will group by the combination of
  * keys and windows.
 
- * <p>See {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}
+ * <p>See {@link org.apache.beam.sdk.transforms.GroupByKey}
  * for more information about how grouping with windows works.
  *
  * <h2> Windowing </h2>
@@ -131,7 +131,7 @@ import javax.annotation.Nullable;
  *      .withAllowedLateness(Duration.ZERO));
  * } </pre>
  *
- * <p>After a {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} the trigger is set to
+ * <p>After a {@link org.apache.beam.sdk.transforms.GroupByKey} the trigger is set to
  * a trigger that will preserve the intent of the upstream trigger.  See
  * {@link Trigger#getContinuationTrigger} for more information.
  *
@@ -161,7 +161,7 @@ public class Window {
    * Creates a {@code Window} {@code PTransform} with the given name.
    *
    * <p>See the discussion of Naming in
-   * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more explanation.
+   * {@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
@@ -259,7 +259,7 @@ public class Window {
      * transform.  The resulting transform is still incomplete.
      *
      * <p>See the discussion of Naming in
-     * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more
+     * {@link org.apache.beam.sdk.transforms.ParDo} for more
      * explanation.
      */
     public Unbound named(String name) {
@@ -282,7 +282,7 @@ public class Window {
      * Elements that are assigned to a specific window will be output when
      * the trigger fires.
      *
-     * <p>{@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger}
+     * <p>{@link org.apache.beam.sdk.transforms.windowing.Trigger}
      * has more details on the available triggers.
      *
      * <p>Must also specify allowed lateness using {@link #withAllowedLateness} and accumulation
@@ -412,7 +412,7 @@ public class Window {
      * modify this {@code PTransform}.
      *
      * <p>See the discussion of Naming in
-     * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more
+     * {@link org.apache.beam.sdk.transforms.ParDo} for more
      * explanation.
      */
     public Bound<T> named(String name) {
@@ -425,7 +425,7 @@ public class Window {
      * Elements that are assigned to a specific window will be output when
      * the trigger fires.
      *
-     * <p>{@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger}
+     * <p>{@link org.apache.beam.sdk.transforms.windowing.Trigger}
      * has more details on the available triggers.
      *
      * <p>Must also specify allowed lateness using {@link #withAllowedLateness} and accumulation
@@ -619,7 +619,7 @@ public class Window {
   /**
    * Creates a {@code Window} {@code PTransform} that does not change assigned
    * windows, but will cause windows to be merged again as part of the next
-   * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}.
+   * {@link org.apache.beam.sdk.transforms.GroupByKey}.
    */
   public static <T> Remerge<T> remerge() {
     return new Remerge<T>();
@@ -628,7 +628,7 @@ public class Window {
   /**
    * {@code PTransform} that does not change assigned windows, but will cause
    *  windows to be merged again as part of the next
-   * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}.
+   * {@link org.apache.beam.sdk.transforms.GroupByKey}.
    */
   private static class Remerge<T> extends PTransform<PCollection<T>, PCollection<T>> {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
index baed3bd..1dda7d0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+
 import com.google.common.collect.Ordering;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
index 3fced0d..406e279 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 /**
- * Defines the {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} transform
+ * Defines the {@link org.apache.beam.sdk.transforms.windowing.Window} transform
  * for dividing the elements in a PCollection into windows, and the
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for controlling when those
+ * {@link org.apache.beam.sdk.transforms.windowing.Trigger} for controlling when those
  * elements are output.
  *
  * <p>{@code Window} logically divides up or groups the elements of a
- * {@link com.google.cloud.dataflow.sdk.values.PCollection} into finite windows according to a
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}.
+ * {@link org.apache.beam.sdk.values.PCollection} into finite windows according to a
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn}.
  * The output of {@code Window} contains the same elements as input, but they
  * have been logically assigned to windows. The next
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}s, including one
+ * {@link org.apache.beam.sdk.transforms.GroupByKey}s, including one
  * within composite transforms, will group by the combination of keys and
  * windows.
  *
@@ -42,9 +42,9 @@
  * <p>As elements are assigned to a window, they are are placed into a pane. When the trigger fires
  * all of the elements in the current pane are output.
  *
- * <p>The {@link com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger} will output a
+ * <p>The {@link org.apache.beam.sdk.transforms.windowing.DefaultTrigger} will output a
  * window when the system watermark passes the end of the window.  See
- * {@link com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark} for details on the
+ * {@link org.apache.beam.sdk.transforms.windowing.AfterWatermark} for details on the
  * watermark.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
index 1782ae3..e1ab9e9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
@@ -15,12 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import java.util.Collection;
 import java.util.Set;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
index 8a2d3f6..e154b58 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Supplier;
@@ -611,10 +611,10 @@ public class ApiSurface {
   ////////////////////////////////////////////////////////////////////////////
 
   public static ApiSurface getSdkApiSurface() throws IOException {
-    return ApiSurface.ofPackage("com.google.cloud.dataflow")
-        .pruningPattern("com[.]google[.]cloud[.]dataflow.*Test")
-        .pruningPattern("com[.]google[.]cloud[.]dataflow.*Benchmark")
-        .pruningPrefix("com.google.cloud.dataflow.integration")
+    return ApiSurface.ofPackage("org.apache.beam")
+        .pruningPattern("org[.]apache[.]beam.*Test")
+        .pruningPattern("org[.]apache[.]beam.*Benchmark")
+        .pruningPrefix("org.apache.beam.integration")
         .pruningPrefix("java")
         .pruningPrefix("com.google.api")
         .pruningPrefix("com.google.auth")

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
index 43daecb..b0fcbd1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppEngineEnvironment.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.lang.reflect.InvocationTargetException;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
index 6ea4a2f..1e7d7c0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.values.PCollectionView;
+
 import com.google.common.annotations.VisibleForTesting;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
index 57f489d..af28052 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
 
 /**
  * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
index f5fd271..e71a47e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 77b4825..833e9b3 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.util.BackOff;
 import com.google.api.client.util.NanoClock;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 2f35d13..cfc53f0 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.util.BackOff;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
index c3bbf9b..df2308d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.MoreObjects.firstNonNull;
 import static com.google.common.base.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
index 1925aa6..33df089 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.worker.StateSampler;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -88,7 +88,7 @@ public abstract class BaseExecutionContext<T extends ExecutionContext.StepContex
 
   /**
    * Hook for subclasses to implement that will be called whenever
-   * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#output}
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
    * is called.
    */
   @Override
@@ -96,7 +96,7 @@ public abstract class BaseExecutionContext<T extends ExecutionContext.StepContex
 
   /**
    * Hook for subclasses to implement that will be called whenever
-   * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#sideOutput}
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
    * is called.
    */
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
index 0311a99..d0c0b2f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b39b2e8..0fecfdc 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
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.BigQueryOptions;
 
 import com.google.api.services.bigquery.model.JobConfigurationLoad;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
 
 import java.io.IOException;
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f0c8864..0502c6e 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
@@ -15,7 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.BigQueryOptions;
 
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.util.BackOff;
@@ -28,7 +30,6 @@ import com.google.api.services.bigquery.model.JobConfigurationLoad;
 import com.google.api.services.bigquery.model.JobReference;
 import com.google.api.services.bigquery.model.JobStatus;
 import com.google.api.services.bigquery.model.TableReference;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 91a0c9e..86a0b5b 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
@@ -15,7 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+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.transforms.Aggregator;
 
 import com.google.api.client.util.BackOff;
 import com.google.api.client.util.BackOffUtils;
@@ -29,10 +34,6 @@ 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.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
index d030870..ca1ac69 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
index 769429d..82c5f51 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.CoderException;
 
 import java.io.IOException;
 import java.io.InputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java
index 914ff8d..6335de4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.Coder.Context;
 
 import java.io.IOException;
 import java.io.OutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java
index a6848f1..ad79b98 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.util.Collections;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java
index 5e2d56d..8d3ec0b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.api.client.util.Preconditions.checkNotNull;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
index 69e949e..67ad54b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addList;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.KvCoderBase;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.MapCoderBase;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.google.api.client.util.Base64;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoderBase;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoderBase;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.base.Throwables;
 
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
@@ -237,7 +238,7 @@ public final class CoderUtils {
      * <p>This resolver resolves coders.  If the Coder ID is a particular
      * well-known identifier supplied by the Dataflow service, it's replaced
      * with the corresponding class.  All other Coder instances are resolved
-     * by class name, using the package com.google.cloud.dataflow.sdk.coders
+     * by class name, using the package org.apache.beam.sdk.coders
      * if there are no "."s in the ID.
      */
     private static final class Resolver extends TypeIdResolverBase {
@@ -284,7 +285,7 @@ public final class CoderUtils {
           }
 
           // Otherwise, see if the ID is the name of a class in
-          // com.google.cloud.dataflow.sdk.coders.  We do this via creating
+          // org.apache.beam.sdk.coders.  We do this via creating
           // the class object so that class loaders have a chance to get
           // involved -- and since we need the class object anyway.
           return Class.forName(Coder.class.getPackage().getName() + "." + id);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
index 6316b64..f73fae3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.state.StateContext;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.values.PCollectionView;
 
 /**
  * Factory that produces {@code Combine.Context} based on different inputs.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java
index ed07efa..fbb683c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.GlobalCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineFnBase.PerKeyCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.util.state.StateContext;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn;
+import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.util.state.StateContext;
 
 import java.io.IOException;
 import java.io.NotSerializableException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
index e5f892b..de0c251 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterProvider;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterProvider;
+import org.apache.beam.sdk.util.common.CounterSet;
 
 /**
  * An implementation of the {@code Aggregator} interface that uses a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
index 40f5e52..3283c8e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.auth.oauth2.Credential;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 5fe9bd9..6fbbcb5 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
@@ -15,7 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.GcpOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.extensions.java6.auth.oauth2.AbstractPromptReceiver;
@@ -30,7 +32,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.cloud.dataflow.sdk.options.GcpOptions;
 import com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DataflowReleaseInfo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DataflowReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DataflowReleaseInfo.java
index 1a06d11..8c096fb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DataflowReleaseInfo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DataflowReleaseInfo.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.json.GenericJson;
 import com.google.api.client.util.Key;
@@ -34,7 +34,7 @@ public final class DataflowReleaseInfo extends GenericJson {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowReleaseInfo.class);
 
   private static final String DATAFLOW_PROPERTIES_PATH =
-      "/com/google/cloud/dataflow/sdk/sdk.properties";
+      "/org.apache.beam/sdk/sdk.properties";
 
   private static class LazyInit {
     private static final DataflowReleaseInfo INSTANCE =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
index d630ece..c3da3d7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.runners.DirectPipelineRunner.ValueWithMetadata;
+import org.apache.beam.sdk.util.common.worker.StateSampler;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java
index a2274dc..f44c06e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnInfo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnInfo.java
index 9de135a..ae19a17 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnInfo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnInfo.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
index 763cd2c..4ec8920 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.ProcessContext;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
+import org.apache.beam.sdk.values.KV;
 
 /**
  * An wrapper interface that represents the execution of a {@link DoFn}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
index 7795f4a..e9202a2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -15,27 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.DoFn.RequiresWindowAccess;
+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.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
index 8a828dd..648a281 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner.ReduceFnExecutor;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 71c8237..9fcd21b 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+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.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
 import com.google.common.base.Preconditions;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
index d576716..577aa66 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.worker.StateSampler;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -44,14 +44,14 @@ public interface ExecutionContext {
 
   /**
    * Hook for subclasses to implement that will be called whenever
-   * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#output}
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
    * is called.
    */
   void noteOutput(WindowedValue<?> output);
 
   /**
    * Hook for subclasses to implement that will be called whenever
-   * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#sideOutput}
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
    * is called.
    */
   void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);
@@ -73,14 +73,14 @@ public interface ExecutionContext {
 
     /**
      * Hook for subclasses to implement that will be called whenever
-     * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#output}
+     * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
      * is called.
      */
     void noteOutput(WindowedValue<?> output);
 
     /**
      * Hook for subclasses to implement that will be called whenever
-     * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#sideOutput}
+     * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
      * is called.
      */
     void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
index 0dd2409..9197fbe 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
index 029c0f5..412c8b6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
index dc2a303..7bc09e9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
index e0f14de..6666ab9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * A mutable set which tracks whether any particular {@link ExecutableTrigger} is

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
index 3195f04..4cd617f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.util.BitSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
index e2313be..23aaa54 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
index 3348d64..98b82e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 
 import java.io.IOException;
 import java.security.GeneralSecurityException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
index 9d21b45..2122c64 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
 import java.io.IOException;
 import java.nio.channels.ReadableByteChannel;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 21c3093..f7ef447 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
@@ -15,7 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
 import com.google.api.client.util.BackOff;
@@ -23,10 +28,6 @@ import com.google.api.client.util.Sleeper;
 import com.google.api.services.storage.Storage;
 import com.google.api.services.storage.model.Objects;
 import com.google.api.services.storage.model.StorageObject;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
 import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel;
 import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
 import com.google.cloud.hadoop.gcsio.ObjectWriteConditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
index 7ab95c7..c24f6da 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner.ReduceFnExecutor;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.KV;
 
 /**
  * A general {@link GroupAlsoByWindowsDoFn}. This delegates all of the logic to the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
index 9990081..f5de0bc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
 
 /**
  * DoFn that merges windows and groups elements in those windows, optionally

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
index 3332480..d364168 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.KV;
 
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.collect.Iterables;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
index aa5b474..d77dd2d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+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.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.util.ArrayList;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
index f4d389a..c89c7ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java
index 3e87a2b..05443fb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IllegalMutationException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IllegalMutationException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IllegalMutationException.java
index 0c7cfc4..b4d0f86 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IllegalMutationException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IllegalMutationException.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 /**
  * Thrown when a value appears to have been mutated, but that mutation is forbidden.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 b3c0389..04ee7b4 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 206c83b..a1d795b 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import com.google.api.client.util.BackOff;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java
index 4efc0d1..b273466 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
 
 /**
  * Interface that contains all the timers and elements associated with a specific work item.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java
index d14e4c5..763f68b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerDataCoder;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+
 import com.google.common.collect.ImmutableList;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java
index 6785c5e..11d84d0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
 
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
index f89a469..4815162 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
 
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;



[17/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java
deleted file mode 100644
index 3ed2d7b..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java
+++ /dev/null
@@ -1,278 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * A {@link Source} that reads a finite amount of input and, because of that, supports
- * some additional operations.
- *
- * <p>The operations are:
- * <ul>
- * <li>Splitting into bundles of given size: {@link #splitIntoBundles};
- * <li>Size estimation: {@link #getEstimatedSizeBytes};
- * <li>Telling whether or not this source produces key/value pairs in sorted order:
- * {@link #producesSortedKeys};
- * <li>The reader ({@link BoundedReader}) supports progress estimation
- * ({@link BoundedReader#getFractionConsumed}) and dynamic splitting
- * ({@link BoundedReader#splitAtFraction}).
- * </ul>
- *
- * <p>To use this class for supporting your custom input type, derive your class
- * class from it, and override the abstract methods. For an example, see {@link DatastoreIO}.
- *
- * @param <T> Type of records read by the source.
- */
-public abstract class BoundedSource<T> extends Source<T> {
-  /**
-   * Splits the source into bundles of approximately {@code desiredBundleSizeBytes}.
-   */
-  public abstract List<? extends BoundedSource<T>> splitIntoBundles(
-      long desiredBundleSizeBytes, PipelineOptions options) throws Exception;
-
-  /**
-   * An estimate of the total size (in bytes) of the data that would be read from this source.
-   * This estimate is in terms of external storage size, before any decompression or other
-   * processing done by the reader.
-   */
-  public abstract long getEstimatedSizeBytes(PipelineOptions options) throws Exception;
-
-  /**
-   * Whether this source is known to produce key/value pairs sorted by lexicographic order on
-   * the bytes of the encoded key.
-   */
-  public abstract boolean producesSortedKeys(PipelineOptions options) throws Exception;
-
-  /**
-   * Returns a new {@link BoundedReader} that reads from this source.
-   */
-  public abstract BoundedReader<T> createReader(PipelineOptions options) throws IOException;
-
-  /**
-   * A {@code Reader} that reads a bounded amount of input and supports some additional
-   * operations, such as progress estimation and dynamic work rebalancing.
-   *
-   * <h3>Boundedness</h3>
-   * <p>Once {@link #start} or {@link #advance} has returned false, neither will be called
-   * again on this object.
-   *
-   * <h3>Thread safety</h3>
-   * All methods will be run from the same thread except {@link #splitAtFraction},
-   * {@link #getFractionConsumed} and {@link #getCurrentSource}, which can be called concurrently
-   * from a different thread. There will not be multiple concurrent calls to
-   * {@link #splitAtFraction} but there can be for {@link #getFractionConsumed} if
-   * {@link #splitAtFraction} is implemented.
-   *
-   * <p>If the source does not implement {@link #splitAtFraction}, you do not need to worry about
-   * thread safety. If implemented, it must be safe to call {@link #splitAtFraction} and
-   * {@link #getFractionConsumed} concurrently with other methods.
-   *
-   * <p>Additionally, a successful {@link #splitAtFraction} call must, by definition, cause
-   * {@link #getCurrentSource} to start returning a different value.
-   * Callers of {@link #getCurrentSource} need to be aware of the possibility that the returned
-   * value can change at any time, and must only access the properties of the source returned by
-   * {@link #getCurrentSource} which do not change between {@link #splitAtFraction} calls.
-   *
-   * <h3>Implementing {@link #splitAtFraction}</h3>
-   * In the course of dynamic work rebalancing, the method {@link #splitAtFraction}
-   * may be called concurrently with {@link #advance} or {@link #start}. It is critical that
-   * their interaction is implemented in a thread-safe way, otherwise data loss is possible.
-   *
-   * <p>Sources which support dynamic work rebalancing should use
-   * {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} to manage the (source-specific)
-   * range of positions that is being split. If your source supports dynamic work rebalancing,
-   * please use that class to implement it if possible; if not possible, please contact the team
-   * at <i>dataflow-feedback@google.com</i>.
-   */
-  @Experimental(Experimental.Kind.SOURCE_SINK)
-  public abstract static class BoundedReader<T> extends Source.Reader<T> {
-    /**
-     * Returns a value in [0, 1] representing approximately what fraction of the
-     * {@link #getCurrentSource current source} this reader has read so far, or {@code null} if such
-     * an estimate is not available.
-     *
-     * <p>It is recommended that this method should satisfy the following properties:
-     * <ul>
-     *   <li>Should return 0 before the {@link #start} call.
-     *   <li>Should return 1 after a {@link #start} or {@link #advance} call that returns false.
-     *   <li>The returned values should be non-decreasing (though they don't have to be unique).
-     * </ul>
-     *
-     * <p>By default, returns null to indicate that this cannot be estimated.
-     *
-     * <h5>Thread safety</h5>
-     * If {@link #splitAtFraction} is implemented, this method can be called concurrently to other
-     * methods (including itself), and it is therefore critical for it to be implemented
-     * in a thread-safe way.
-     */
-    public Double getFractionConsumed() {
-      return null;
-    }
-
-    /**
-     * Returns a {@code Source} describing the same input that this {@code Reader} currently reads
-     * (including items already read).
-     *
-     * <h3>Usage</h3>
-     * <p>Reader subclasses can use this method for convenience to access unchanging properties of
-     * the source being read. Alternatively, they can cache these properties in the constructor.
-     * <p>The framework will call this method in the course of dynamic work rebalancing, e.g. after
-     * a successful {@link BoundedSource.BoundedReader#splitAtFraction} call.
-     *
-     * <h3>Mutability and thread safety</h3>
-     * Remember that {@link Source} objects must always be immutable. However, the return value of
-     * this function may be affected by dynamic work rebalancing, happening asynchronously via
-     * {@link BoundedSource.BoundedReader#splitAtFraction}, meaning it can return a different
-     * {@link Source} object. However, the returned object itself will still itself be immutable.
-     * Callers must take care not to rely on properties of the returned source that may be
-     * asynchronously changed as a result of this process (e.g. do not cache an end offset when
-     * reading a file).
-     *
-     * <h3>Implementation</h3>
-     * For convenience, subclasses should usually return the most concrete subclass of
-     * {@link Source} possible.
-     * In practice, the implementation of this method should nearly always be one of the following:
-     * <ul>
-     *   <li>Source that inherits from a base class that already implements
-     *   {@link #getCurrentSource}: delegate to base class. In this case, it is almost always
-     *   an error for the subclass to maintain its own copy of the source.
-     * <pre>{@code
-     *   public FooReader(FooSource<T> source) {
-     *     super(source);
-     *   }
-     *
-     *   public FooSource<T> getCurrentSource() {
-     *     return (FooSource<T>)super.getCurrentSource();
-     *   }
-     * }</pre>
-     *   <li>Source that does not support dynamic work rebalancing: return a private final variable.
-     * <pre>{@code
-     *   private final FooSource<T> source;
-     *
-     *   public FooReader(FooSource<T> source) {
-     *     this.source = source;
-     *   }
-     *
-     *   public FooSource<T> getCurrentSource() {
-     *     return source;
-     *   }
-     * }</pre>
-     *   <li>{@link BoundedSource.BoundedReader} that explicitly supports dynamic work rebalancing:
-     *   maintain a variable pointing to an immutable source object, and protect it with
-     *   synchronization.
-     * <pre>{@code
-     *   private FooSource<T> source;
-     *
-     *   public FooReader(FooSource<T> source) {
-     *     this.source = source;
-     *   }
-     *
-     *   public synchronized FooSource<T> getCurrentSource() {
-     *     return source;
-     *   }
-     *
-     *   public synchronized FooSource<T> splitAtFraction(double fraction) {
-     *     ...
-     *     FooSource<T> primary = ...;
-     *     FooSource<T> residual = ...;
-     *     this.source = primary;
-     *     return residual;
-     *   }
-     * }</pre>
-     * </ul>
-     */
-    @Override
-    public abstract BoundedSource<T> getCurrentSource();
-
-    /**
-     * Tells the reader to narrow the range of the input it's going to read and give up
-     * the remainder, so that the new range would contain approximately the given
-     * fraction of the amount of data in the current range.
-     *
-     * <p>Returns a {@code BoundedSource} representing the remainder.
-     *
-     * <h5>Detailed description</h5>
-     * Assuming the following sequence of calls:
-     * <pre>{@code
-     *   BoundedSource<T> initial = reader.getCurrentSource();
-     *   BoundedSource<T> residual = reader.splitAtFraction(fraction);
-     *   BoundedSource<T> primary = reader.getCurrentSource();
-     * }</pre>
-     * <ul>
-     *  <li> The "primary" and "residual" sources, when read, should together cover the same
-     *  set of records as "initial".
-     *  <li> The current reader should continue to be in a valid state, and continuing to read
-     *  from it should, together with the records it already read, yield the same records
-     *  as would have been read by "primary".
-     *  <li> The amount of data read by "primary" should ideally represent approximately
-     *  the given fraction of the amount of data read by "initial".
-     * </ul>
-     * For example, a reader that reads a range of offsets <i>[A, B)</i> in a file might implement
-     * this method by truncating the current range to <i>[A, A + fraction*(B-A))</i> and returning
-     * a Source representing the range <i>[A + fraction*(B-A), B)</i>.
-     *
-     * <p>This method should return {@code null} if the split cannot be performed for this fraction
-     * while satisfying the semantics above. E.g., a reader that reads a range of offsets
-     * in a file should return {@code null} if it is already past the position in its range
-     * corresponding to the given fraction. In this case, the method MUST have no effect
-     * (the reader must behave as if the method hadn't been called at all).
-     *
-     * <h5>Statefulness</h5>
-     * Since this method (if successful) affects the reader's source, in subsequent invocations
-     * "fraction" should be interpreted relative to the new current source.
-     *
-     * <h5>Thread safety and blocking</h5>
-     * This method will be called concurrently to other methods (however there will not be multiple
-     * concurrent invocations of this method itself), and it is critical for it to be implemented
-     * in a thread-safe way (otherwise data loss is possible).
-     *
-     * <p>It is also very important that this method always completes quickly. In particular,
-     * it should not perform or wait on any blocking operations such as I/O, RPCs etc. Violating
-     * this requirement may stall completion of the work item or even cause it to fail.
-     *
-     * <p>It is incorrect to make both this method and {@link #start}/{@link #advance}
-     * {@code synchronized}, because those methods can perform blocking operations, and then
-     * this method would have to wait for those calls to complete.
-     *
-     * <p>{@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} makes it easy to implement
-     * this method safely and correctly.
-     *
-     * <p>By default, returns null to indicate that splitting is not possible.
-     */
-    public BoundedSource<T> splitAtFraction(double fraction) {
-      return null;
-    }
-
-    /**
-     * By default, returns the minimum possible timestamp.
-     */
-    @Override
-    public Instant getCurrentTimestamp() throws NoSuchElementException {
-      return BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java
deleted file mode 100644
index 9543749..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java
+++ /dev/null
@@ -1,414 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-import com.google.common.io.ByteStreams;
-import com.google.common.primitives.Ints;
-
-import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
-import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
-
-import java.io.IOException;
-import java.io.PushbackInputStream;
-import java.io.Serializable;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.util.NoSuchElementException;
-import java.util.zip.GZIPInputStream;
-
-/**
- * A Source that reads from compressed files. A {@code CompressedSources} wraps a delegate
- * {@link FileBasedSource} that is able to read the decompressed file format.
- *
- * <p>For example, use the following to read from a gzip-compressed XML file:
- *
- * <pre> {@code
- * XmlSource mySource = XmlSource.from(...);
- * PCollection<T> collection = p.apply(Read.from(CompressedSource
- *     .from(mySource)
- *     .withDecompression(CompressedSource.CompressionMode.GZIP)));
- * } </pre>
- *
- * <p>Supported compression algorithms are {@link CompressionMode#GZIP} and
- * {@link CompressionMode#BZIP2}. User-defined compression types are supported by implementing
- * {@link DecompressingChannelFactory}.
- *
- * <p>By default, the compression algorithm is selected from those supported in
- * {@link CompressionMode} based on the file name provided to the source, namely
- * {@code ".bz2"} indicates {@link CompressionMode#BZIP2} and {@code ".gz"} indicates
- * {@link CompressionMode#GZIP}. If the file name does not match any of the supported
- * algorithms, it is assumed to be uncompressed data.
- *
- * @param <T> The type to read from the compressed file.
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public class CompressedSource<T> extends FileBasedSource<T> {
-  /**
-   * Factory interface for creating channels that decompress the content of an underlying channel.
-   */
-  public static interface DecompressingChannelFactory extends Serializable {
-    /**
-     * Given a channel, create a channel that decompresses the content read from the channel.
-     * @throws IOException
-     */
-    public ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel)
-        throws IOException;
-  }
-
-  /**
-   * Factory interface for creating channels that decompress the content of an underlying channel,
-   * based on both the channel and the file name.
-   */
-  private static interface FileNameBasedDecompressingChannelFactory
-      extends DecompressingChannelFactory {
-    /**
-     * Given a channel, create a channel that decompresses the content read from the channel.
-     * @throws IOException
-     */
-    ReadableByteChannel createDecompressingChannel(String fileName, ReadableByteChannel channel)
-        throws IOException;
-
-    /**
-     * Given a file name, returns true if the file name matches any supported compression
-     * scheme.
-     */
-    boolean isCompressed(String fileName);
-  }
-
-  /**
-   * Default compression types supported by the {@code CompressedSource}.
-   */
-  public enum CompressionMode implements DecompressingChannelFactory {
-    /**
-     * Reads a byte channel assuming it is compressed with gzip.
-     */
-    GZIP {
-      @Override
-      public boolean matches(String fileName) {
-          return fileName.toLowerCase().endsWith(".gz");
-      }
-
-      @Override
-      public ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel)
-          throws IOException {
-        // Determine if the input stream is gzipped. The input stream returned from the
-        // GCS connector may already be decompressed; GCS does this based on the
-        // content-encoding property.
-        PushbackInputStream stream = new PushbackInputStream(Channels.newInputStream(channel), 2);
-        byte[] headerBytes = new byte[2];
-        int bytesRead = ByteStreams.read(
-            stream /* source */, headerBytes /* dest */, 0 /* offset */, 2 /* len */);
-        stream.unread(headerBytes, 0, bytesRead);
-        if (bytesRead >= 2) {
-          byte zero = 0x00;
-          int header = Ints.fromBytes(zero, zero, headerBytes[1], headerBytes[0]);
-          if (header == GZIPInputStream.GZIP_MAGIC) {
-            return Channels.newChannel(new GzipCompressorInputStream(stream, true));
-          }
-        }
-        return Channels.newChannel(stream);
-      }
-    },
-
-    /**
-     * Reads a byte channel assuming it is compressed with bzip2.
-     */
-    BZIP2 {
-      @Override
-      public boolean matches(String fileName) {
-          return fileName.toLowerCase().endsWith(".bz2");
-      }
-
-      @Override
-      public ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel)
-          throws IOException {
-        return Channels.newChannel(
-            new BZip2CompressorInputStream(Channels.newInputStream(channel)));
-      }
-    };
-
-    /**
-     * Returns {@code true} if the given file name implies that the contents are compressed
-     * according to the compression embodied by this factory.
-     */
-    public abstract boolean matches(String fileName);
-
-    @Override
-    public abstract ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel)
-        throws IOException;
-  }
-
-  /**
-   * Reads a byte channel detecting compression according to the file name. If the filename
-   * is not any other known {@link CompressionMode}, it is presumed to be uncompressed.
-   */
-  private static class DecompressAccordingToFilename
-      implements FileNameBasedDecompressingChannelFactory {
-
-    @Override
-    public ReadableByteChannel createDecompressingChannel(
-        String fileName, ReadableByteChannel channel) throws IOException {
-      for (CompressionMode type : CompressionMode.values()) {
-        if (type.matches(fileName)) {
-          return type.createDecompressingChannel(channel);
-        }
-      }
-      // Uncompressed
-      return channel;
-    }
-
-    @Override
-    public ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel) {
-      throw new UnsupportedOperationException(
-          String.format("%s does not support createDecompressingChannel(%s) but only"
-              + " createDecompressingChannel(%s,%s)",
-              getClass().getSimpleName(),
-              String.class.getSimpleName(),
-              ReadableByteChannel.class.getSimpleName(),
-              ReadableByteChannel.class.getSimpleName()));
-    }
-
-    @Override
-    public boolean isCompressed(String fileName) {
-      for (CompressionMode type : CompressionMode.values()) {
-        if  (type.matches(fileName)) {
-          return true;
-        }
-      }
-      return false;
-    }
-  }
-
-  private final FileBasedSource<T> sourceDelegate;
-  private final DecompressingChannelFactory channelFactory;
-
-  /**
-   * Creates a {@link Read} transform that reads from that reads from the underlying
-   * {@link FileBasedSource} {@code sourceDelegate} after decompressing it with a {@link
-   * DecompressingChannelFactory}.
-   */
-  public static <T> Read.Bounded<T> readFromSource(
-      FileBasedSource<T> sourceDelegate, DecompressingChannelFactory channelFactory) {
-    return Read.from(new CompressedSource<>(sourceDelegate, channelFactory));
-  }
-
-  /**
-   * Creates a {@code CompressedSource} from an underlying {@code FileBasedSource}. The type
-   * of compression used will be based on the file name extension unless explicitly
-   * configured via {@link CompressedSource#withDecompression}.
-   */
-  public static <T> CompressedSource<T> from(FileBasedSource<T> sourceDelegate) {
-    return new CompressedSource<>(sourceDelegate, new DecompressAccordingToFilename());
-  }
-
-  /**
-   * Return a {@code CompressedSource} that is like this one but will decompress its underlying file
-   * with the given {@link DecompressingChannelFactory}.
-   */
-  public CompressedSource<T> withDecompression(DecompressingChannelFactory channelFactory) {
-    return new CompressedSource<>(this.sourceDelegate, channelFactory);
-  }
-
-  /**
-   * Creates a {@code CompressedSource} from a delegate file based source and a decompressing
-   * channel factory.
-   */
-  private CompressedSource(
-      FileBasedSource<T> sourceDelegate, DecompressingChannelFactory channelFactory) {
-    super(sourceDelegate.getFileOrPatternSpec(), Long.MAX_VALUE);
-    this.sourceDelegate = sourceDelegate;
-    this.channelFactory = channelFactory;
-  }
-
-  /**
-   * Creates a {@code CompressedSource} for an individual file. Used by {@link
-   * CompressedSource#createForSubrangeOfFile}.
-   */
-  private CompressedSource(FileBasedSource<T> sourceDelegate,
-      DecompressingChannelFactory channelFactory, String filePatternOrSpec, long minBundleSize,
-      long startOffset, long endOffset) {
-    super(filePatternOrSpec, minBundleSize, startOffset, endOffset);
-    Preconditions.checkArgument(
-        startOffset == 0,
-        "CompressedSources must start reading at offset 0. Requested offset: " + startOffset);
-    this.sourceDelegate = sourceDelegate;
-    this.channelFactory = channelFactory;
-  }
-
-  /**
-   * Validates that the delegate source is a valid source and that the channel factory is not null.
-   */
-  @Override
-  public void validate() {
-    super.validate();
-    Preconditions.checkNotNull(sourceDelegate);
-    sourceDelegate.validate();
-    Preconditions.checkNotNull(channelFactory);
-  }
-
-  /**
-   * Creates a {@code CompressedSource} for a subrange of a file. Called by superclass to create a
-   * source for a single file.
-   */
-  @Override
-  protected FileBasedSource<T> createForSubrangeOfFile(String fileName, long start, long end) {
-    return new CompressedSource<>(sourceDelegate.createForSubrangeOfFile(fileName, start, end),
-        channelFactory, fileName, Long.MAX_VALUE, start, end);
-  }
-
-  /**
-   * Determines whether a single file represented by this source is splittable. Returns true
-   * if we are using the default decompression factory and and it determines
-   * from the requested file name that the file is not compressed.
-   */
-  @Override
-  protected final boolean isSplittable() throws Exception {
-    if (channelFactory instanceof FileNameBasedDecompressingChannelFactory) {
-      FileNameBasedDecompressingChannelFactory fileNameBasedChannelFactory =
-          (FileNameBasedDecompressingChannelFactory) channelFactory;
-      return !fileNameBasedChannelFactory.isCompressed(getFileOrPatternSpec());
-    }
-    return true;
-  }
-
-  /**
-   * Creates a {@code FileBasedReader} to read a single file.
-   *
-   * <p>Uses the delegate source to create a single file reader for the delegate source.
-   * Utilizes the default decompression channel factory to not wrap the source reader
-   * if the file name does not represent a compressed file allowing for splitting of
-   * the source.
-   */
-  @Override
-  protected final FileBasedReader<T> createSingleFileReader(PipelineOptions options) {
-    if (channelFactory instanceof FileNameBasedDecompressingChannelFactory) {
-      FileNameBasedDecompressingChannelFactory fileNameBasedChannelFactory =
-          (FileNameBasedDecompressingChannelFactory) channelFactory;
-      if (!fileNameBasedChannelFactory.isCompressed(getFileOrPatternSpec())) {
-        return sourceDelegate.createSingleFileReader(options);
-      }
-    }
-    return new CompressedReader<T>(
-        this, sourceDelegate.createSingleFileReader(options));
-  }
-
-  /**
-   * Returns whether the delegate source produces sorted keys.
-   */
-  @Override
-  public final boolean producesSortedKeys(PipelineOptions options) throws Exception {
-    return sourceDelegate.producesSortedKeys(options);
-  }
-
-  /**
-   * Returns the delegate source's default output coder.
-   */
-  @Override
-  public final Coder<T> getDefaultOutputCoder() {
-    return sourceDelegate.getDefaultOutputCoder();
-  }
-
-  public final DecompressingChannelFactory getChannelFactory() {
-    return channelFactory;
-  }
-
-  /**
-   * Reader for a {@link CompressedSource}. Decompresses its input and uses a delegate
-   * reader to read elements from the decompressed input.
-   * @param <T> The type of records read from the source.
-   */
-  public static class CompressedReader<T> extends FileBasedReader<T> {
-
-    private final FileBasedReader<T> readerDelegate;
-    private final CompressedSource<T> source;
-    private int numRecordsRead;
-
-    /**
-     * Create a {@code CompressedReader} from a {@code CompressedSource} and delegate reader.
-     */
-    public CompressedReader(CompressedSource<T> source, FileBasedReader<T> readerDelegate) {
-      super(source);
-      this.source = source;
-      this.readerDelegate = readerDelegate;
-    }
-
-    /**
-     * Gets the current record from the delegate reader.
-     */
-    @Override
-    public T getCurrent() throws NoSuchElementException {
-      return readerDelegate.getCurrent();
-    }
-
-    /**
-     * Returns true only for the first record; compressed sources cannot be split.
-     */
-    @Override
-    protected final boolean isAtSplitPoint() {
-      // We have to return true for the first record, but not for the state before reading it,
-      // and not for the state after reading any other record. Hence == rather than >= or <=.
-      // This is required because FileBasedReader is intended for readers that can read a range
-      // of offsets in a file and where the range can be split in parts. CompressedReader,
-      // however, is a degenerate case because it cannot be split, but it has to satisfy the
-      // semantics of offsets and split points anyway.
-      return numRecordsRead == 1;
-    }
-
-    /**
-     * Creates a decompressing channel from the input channel and passes it to its delegate reader's
-     * {@link FileBasedReader#startReading(ReadableByteChannel)}.
-     */
-    @Override
-    protected final void startReading(ReadableByteChannel channel) throws IOException {
-      if (source.getChannelFactory() instanceof FileNameBasedDecompressingChannelFactory) {
-        FileNameBasedDecompressingChannelFactory channelFactory =
-            (FileNameBasedDecompressingChannelFactory) source.getChannelFactory();
-        readerDelegate.startReading(channelFactory.createDecompressingChannel(
-            getCurrentSource().getFileOrPatternSpec(),
-            channel));
-      } else {
-        readerDelegate.startReading(source.getChannelFactory().createDecompressingChannel(
-            channel));
-      }
-    }
-
-    /**
-     * Reads the next record via the delegate reader.
-     */
-    @Override
-    protected final boolean readNextRecord() throws IOException {
-      if (!readerDelegate.readNextRecord()) {
-        return false;
-      }
-      ++numRecordsRead;
-      return true;
-    }
-
-    /**
-     * Returns the delegate reader's current offset in the decompressed input.
-     */
-    @Override
-    protected final long getCurrentOffset() {
-      return readerDelegate.getCurrentOffset();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java
deleted file mode 100644
index 892682d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java
+++ /dev/null
@@ -1,224 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.io.CountingSource.NowTimestampFn;
-import com.google.cloud.dataflow.sdk.io.Read.Unbounded;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.common.base.Optional;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A {@link PTransform} that produces longs. When used to produce a
- * {@link IsBounded#BOUNDED bounded} {@link PCollection}, {@link CountingInput} starts at {@code 0}
- * and counts up to a specified maximum. When used to produce an
- * {@link IsBounded#UNBOUNDED unbounded} {@link PCollection}, it counts up to {@link Long#MAX_VALUE}
- * and then never produces more output. (In practice, this limit should never be reached.)
- *
- * <p>The bounded {@link CountingInput} is implemented based on {@link OffsetBasedSource} and
- * {@link OffsetBasedSource.OffsetBasedReader}, so it performs efficient initial splitting and it
- * supports dynamic work rebalancing.
- *
- * <p>To produce a bounded {@code PCollection<Long>}, use {@link CountingInput#upTo(long)}:
- *
- * <pre>{@code
- * Pipeline p = ...
- * PTransform<PBegin, PCollection<Long>> producer = CountingInput.upTo(1000);
- * PCollection<Long> bounded = p.apply(producer);
- * }</pre>
- *
- * <p>To produce an unbounded {@code PCollection<Long>}, use {@link CountingInput#unbounded()},
- * calling {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to provide values
- * with timestamps other than {@link Instant#now}.
- *
- * <pre>{@code
- * Pipeline p = ...
- *
- * // To create an unbounded producer that uses processing time as the element timestamp.
- * PCollection<Long> unbounded = p.apply(CountingInput.unbounded());
- * // Or, to create an unbounded source that uses a provided function to set the element timestamp.
- * PCollection<Long> unboundedWithTimestamps =
- *     p.apply(CountingInput.unbounded().withTimestampFn(someFn));
- * }</pre>
- */
-public class CountingInput {
-  /**
-   * Creates a {@link BoundedCountingInput} that will produce the specified number of elements,
-   * from {@code 0} to {@code numElements - 1}.
-   */
-  public static BoundedCountingInput upTo(long numElements) {
-    checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements);
-    return new BoundedCountingInput(numElements);
-  }
-
-  /**
-   * Creates an {@link UnboundedCountingInput} that will produce numbers starting from {@code 0} up
-   * to {@link Long#MAX_VALUE}.
-   *
-   * <p>After {@link Long#MAX_VALUE}, the transform never produces more output. (In practice, this
-   * limit should never be reached.)
-   *
-   * <p>Elements in the resulting {@link PCollection PCollection&lt;Long&gt;} will by default have
-   * timestamps corresponding to processing time at element generation, provided by
-   * {@link Instant#now}. Use the transform returned by
-   * {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to control the output
-   * timestamps.
-   */
-  public static UnboundedCountingInput unbounded() {
-    return new UnboundedCountingInput(
-        new NowTimestampFn(),
-        1L /* Elements per period */,
-        Duration.ZERO /* Period length */,
-        Optional.<Long>absent() /* Maximum number of records */,
-        Optional.<Duration>absent() /* Maximum read duration */);
-  }
-
-  /**
-   * A {@link PTransform} that will produce a specified number of {@link Long Longs} starting from
-   * 0.
-   */
-  public static class BoundedCountingInput extends PTransform<PBegin, PCollection<Long>> {
-    private final long numElements;
-
-    private BoundedCountingInput(long numElements) {
-      this.numElements = numElements;
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    public PCollection<Long> apply(PBegin begin) {
-      return begin.apply(Read.from(CountingSource.upTo(numElements)));
-    }
-  }
-
-  /**
-   * A {@link PTransform} that will produce numbers starting from {@code 0} up to
-   * {@link Long#MAX_VALUE}.
-   *
-   * <p>After {@link Long#MAX_VALUE}, the transform never produces more output. (In practice, this
-   * limit should never be reached.)
-   *
-   * <p>Elements in the resulting {@link PCollection PCollection&lt;Long&gt;} will by default have
-   * timestamps corresponding to processing time at element generation, provided by
-   * {@link Instant#now}. Use the transform returned by
-   * {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to control the output
-   * timestamps.
-   */
-  public static class UnboundedCountingInput extends PTransform<PBegin, PCollection<Long>> {
-    private final SerializableFunction<Long, Instant> timestampFn;
-    private final long elementsPerPeriod;
-    private final Duration period;
-    private final Optional<Long> maxNumRecords;
-    private final Optional<Duration> maxReadTime;
-
-    private UnboundedCountingInput(
-        SerializableFunction<Long, Instant> timestampFn,
-        long elementsPerPeriod,
-        Duration period,
-        Optional<Long> maxNumRecords,
-        Optional<Duration> maxReadTime) {
-      this.timestampFn = timestampFn;
-      this.elementsPerPeriod = elementsPerPeriod;
-      this.period = period;
-      this.maxNumRecords = maxNumRecords;
-      this.maxReadTime = maxReadTime;
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingInput} like this one, but where output elements have the
-     * timestamp specified by the timestampFn.
-     *
-     * <p>Note that the timestamps produced by {@code timestampFn} may not decrease.
-     */
-    public UnboundedCountingInput withTimestampFn(SerializableFunction<Long, Instant> timestampFn) {
-      return new UnboundedCountingInput(
-          timestampFn, elementsPerPeriod, period, maxNumRecords, maxReadTime);
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingInput} like this one, but that will read at most the
-     * specified number of elements.
-     *
-     * <p>A bounded amount of elements will be produced by the result transform, and the result
-     * {@link PCollection} will be {@link IsBounded#BOUNDED bounded}.
-     */
-    public UnboundedCountingInput withMaxNumRecords(long maxRecords) {
-      checkArgument(
-          maxRecords > 0, "MaxRecords must be a positive (nonzero) value. Got %s", maxRecords);
-      return new UnboundedCountingInput(
-          timestampFn, elementsPerPeriod, period, Optional.of(maxRecords), maxReadTime);
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingInput} like this one, but with output production limited
-     * to an aggregate rate of no more than the number of elements per the period length.
-     *
-     * <p>Note that when there are multiple splits, each split outputs independently. This may lead
-     * to elements not being produced evenly across time, though the aggregate rate will still
-     * approach the specified rate.
-     *
-     * <p>A duration of {@link Duration#ZERO} will produce output as fast as possible.
-     */
-    public UnboundedCountingInput withRate(long numElements, Duration periodLength) {
-      return new UnboundedCountingInput(
-          timestampFn, numElements, periodLength, maxNumRecords, maxReadTime);
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingInput} like this one, but that will read for at most the
-     * specified amount of time.
-     *
-     * <p>A bounded amount of elements will be produced by the result transform, and the result
-     * {@link PCollection} will be {@link IsBounded#BOUNDED bounded}.
-     */
-    public UnboundedCountingInput withMaxReadTime(Duration readTime) {
-      checkNotNull(readTime, "ReadTime cannot be null");
-      return new UnboundedCountingInput(
-          timestampFn, elementsPerPeriod, period, maxNumRecords, Optional.of(readTime));
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    public PCollection<Long> apply(PBegin begin) {
-      Unbounded<Long> read =
-          Read.from(
-              CountingSource.createUnbounded()
-                  .withTimestampFn(timestampFn)
-                  .withRate(elementsPerPeriod, period));
-      if (!maxNumRecords.isPresent() && !maxReadTime.isPresent()) {
-        return begin.apply(read);
-      } else if (maxNumRecords.isPresent() && !maxReadTime.isPresent()) {
-        return begin.apply(read.withMaxNumRecords(maxNumRecords.get()));
-      } else if (!maxNumRecords.isPresent() && maxReadTime.isPresent()) {
-        return begin.apply(read.withMaxReadTime(maxReadTime.get()));
-      } else {
-        return begin.apply(
-            read.withMaxReadTime(maxReadTime.get()).withMaxNumRecords(maxNumRecords.get()));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java
deleted file mode 100644
index 78c98cb..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java
+++ /dev/null
@@ -1,481 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.CountingInput.UnboundedCountingInput;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * A source that produces longs. When used as a {@link BoundedSource}, {@link CountingSource}
- * starts at {@code 0} and counts up to a specified maximum. When used as an
- * {@link UnboundedSource}, it counts up to {@link Long#MAX_VALUE} and then never produces more
- * output. (In practice, this limit should never be reached.)
- *
- * <p>The bounded {@link CountingSource} is implemented based on {@link OffsetBasedSource} and
- * {@link OffsetBasedSource.OffsetBasedReader}, so it performs efficient initial splitting and it
- * supports dynamic work rebalancing.
- *
- * <p>To produce a bounded {@code PCollection<Long>}, use {@link CountingSource#upTo(long)}:
- *
- * <pre>{@code
- * Pipeline p = ...
- * PTransform<PBegin, PCollection<Long>> producer = CountingInput.upTo(1000);
- * PCollection<Long> bounded = p.apply(producer);
- * }</pre>
- *
- * <p>To produce an unbounded {@code PCollection<Long>}, use {@link CountingInput#unbounded()},
- * calling {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to provide values
- * with timestamps other than {@link Instant#now}.
- *
- * <pre>{@code
- * Pipeline p = ...
- *
- * // To create an unbounded PCollection that uses processing time as the element timestamp.
- * PCollection<Long> unbounded = p.apply(CountingInput.unbounded());
- * // Or, to create an unbounded source that uses a provided function to set the element timestamp.
- * PCollection<Long> unboundedWithTimestamps =
- *     p.apply(CountingInput.unbounded().withTimestampFn(someFn));
- *
- * }</pre>
- */
-public class CountingSource {
-  /**
-   * Creates a {@link BoundedSource} that will produce the specified number of elements,
-   * from {@code 0} to {@code numElements - 1}.
-   *
-   * @deprecated use {@link CountingInput#upTo(long)} instead
-   */
-  @Deprecated
-  public static BoundedSource<Long> upTo(long numElements) {
-    checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements);
-    return new BoundedCountingSource(0, numElements);
-  }
-
-  /**
-   * Create a new {@link UnboundedCountingSource}.
-   */
-  // package-private to return a typed UnboundedCountingSource rather than the UnboundedSource type.
-  static UnboundedCountingSource createUnbounded() {
-    return new UnboundedCountingSource(0, 1, 1L, Duration.ZERO, new NowTimestampFn());
-  }
-
-  /**
-   * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to
-   * {@link Long#MAX_VALUE}.
-   *
-   * <p>After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this
-   * limit should never be reached.)
-   *
-   * <p>Elements in the resulting {@link PCollection PCollection&lt;Long&gt;} will have timestamps
-   * corresponding to processing time at element generation, provided by {@link Instant#now}.
-   *
-   * @deprecated use {@link CountingInput#unbounded()} instead
-   */
-  @Deprecated
-  public static UnboundedSource<Long, CounterMark> unbounded() {
-    return unboundedWithTimestampFn(new NowTimestampFn());
-  }
-
-  /**
-   * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to
-   * {@link Long#MAX_VALUE}, with element timestamps supplied by the specified function.
-   *
-   * <p>After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this
-   * limit should never be reached.)
-   *
-   * <p>Note that the timestamps produced by {@code timestampFn} may not decrease.
-   *
-   * @deprecated use {@link CountingInput#unbounded()} and call
-   *             {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} instead
-   */
-  @Deprecated
-  public static UnboundedSource<Long, CounterMark> unboundedWithTimestampFn(
-      SerializableFunction<Long, Instant> timestampFn) {
-    return new UnboundedCountingSource(0, 1, 1L, Duration.ZERO, timestampFn);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////////////////////
-
-  /** Prevent instantiation. */
-  private CountingSource() {}
-
-  /**
-   * A function that returns {@link Instant#now} as the timestamp for each generated element.
-   */
-  static class NowTimestampFn implements SerializableFunction<Long, Instant> {
-    @Override
-    public Instant apply(Long input) {
-      return Instant.now();
-    }
-  }
-
-  /**
-   * An implementation of {@link CountingSource} that produces a bounded {@link PCollection}.
-   * It is implemented on top of {@link OffsetBasedSource} (with associated reader
-   * {@link BoundedCountingReader}) and performs efficient initial splitting and supports dynamic
-   * work rebalancing.
-   */
-  private static class BoundedCountingSource extends OffsetBasedSource<Long> {
-    /**
-     * Creates a {@link BoundedCountingSource} that generates the numbers in the specified
-     * {@code [start, end)} range.
-     */
-    public BoundedCountingSource(long start, long end) {
-      super(start, end, 1 /* can be split every 1 offset */);
-    }
-
-    ////////////////////////////////////////////////////////////////////////////////////////////
-
-    @Override
-    public long getBytesPerOffset() {
-      return 8;
-    }
-
-    @Override
-    public long getMaxEndOffset(PipelineOptions options) throws Exception {
-      return getEndOffset();
-    }
-
-    @Override
-    public OffsetBasedSource<Long> createSourceForSubrange(long start, long end) {
-      return new BoundedCountingSource(start, end);
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-      return true;
-    }
-
-    @Override
-    public com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader<Long> createReader(
-        PipelineOptions options) throws IOException {
-      return new BoundedCountingReader(this);
-    }
-
-    @Override
-    public Coder<Long> getDefaultOutputCoder() {
-      return VarLongCoder.of();
-    }
-  }
-
-  /**
-   * The reader associated with {@link BoundedCountingSource}.
-   *
-   * @see BoundedCountingSource
-   */
-  private static class BoundedCountingReader extends OffsetBasedSource.OffsetBasedReader<Long> {
-    private long current;
-
-    public BoundedCountingReader(OffsetBasedSource<Long> source) {
-      super(source);
-    }
-
-    @Override
-    protected long getCurrentOffset() throws NoSuchElementException {
-      return current;
-    }
-
-    @Override
-    public synchronized BoundedCountingSource getCurrentSource()  {
-      return (BoundedCountingSource) super.getCurrentSource();
-    }
-
-    @Override
-    public Long getCurrent() throws NoSuchElementException {
-      return current;
-    }
-
-    @Override
-    protected boolean startImpl() throws IOException {
-      current = getCurrentSource().getStartOffset();
-      return true;
-    }
-
-    @Override
-    protected boolean advanceImpl() throws IOException {
-      current++;
-      return true;
-    }
-
-    @Override
-    public void close() throws IOException {}
-  }
-
-  /**
-   * An implementation of {@link CountingSource} that produces an unbounded {@link PCollection}.
-   */
-  static class UnboundedCountingSource extends UnboundedSource<Long, CounterMark> {
-    /** The first number (>= 0) generated by this {@link UnboundedCountingSource}. */
-    private final long start;
-    /** The interval between numbers generated by this {@link UnboundedCountingSource}. */
-    private final long stride;
-    /** The number of elements to produce each period. */
-    private final long elementsPerPeriod;
-    /** The time between producing numbers from this {@link UnboundedCountingSource}. */
-    private final Duration period;
-    /** The function used to produce timestamps for the generated elements. */
-    private final SerializableFunction<Long, Instant> timestampFn;
-
-    /**
-     * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to
-     * {@link Long#MAX_VALUE}, with element timestamps supplied by the specified function.
-     *
-     * <p>After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this
-     * limit should never be reached.)
-     *
-     * <p>Note that the timestamps produced by {@code timestampFn} may not decrease.
-     */
-    private UnboundedCountingSource(
-        long start,
-        long stride,
-        long elementsPerPeriod,
-        Duration period,
-        SerializableFunction<Long, Instant> timestampFn) {
-      this.start = start;
-      this.stride = stride;
-      checkArgument(
-          elementsPerPeriod > 0L,
-          "Must produce at least one element per period, got %s",
-          elementsPerPeriod);
-      this.elementsPerPeriod = elementsPerPeriod;
-      checkArgument(
-          period.getMillis() >= 0L, "Must have a non-negative period length, got %s", period);
-      this.period = period;
-      this.timestampFn = timestampFn;
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingSource} like this one with the specified period. Elements
-     * will be produced with an interval between them equal to the period.
-     */
-    public UnboundedCountingSource withRate(long elementsPerPeriod, Duration period) {
-      return new UnboundedCountingSource(start, stride, elementsPerPeriod, period, timestampFn);
-    }
-
-    /**
-     * Returns an {@link UnboundedCountingSource} like this one where the timestamp of output
-     * elements are supplied by the specified function.
-     *
-     * <p>Note that timestamps produced by {@code timestampFn} may not decrease.
-     */
-    public UnboundedCountingSource withTimestampFn(
-        SerializableFunction<Long, Instant> timestampFn) {
-      checkNotNull(timestampFn);
-      return new UnboundedCountingSource(start, stride, elementsPerPeriod, period, timestampFn);
-    }
-
-    /**
-     * Splits an unbounded source {@code desiredNumSplits} ways by giving each split every
-     * {@code desiredNumSplits}th element that this {@link UnboundedCountingSource}
-     * produces.
-     *
-     * <p>E.g., if a source produces all even numbers {@code [0, 2, 4, 6, 8, ...)} and we want to
-     * split into 3 new sources, then the new sources will produce numbers that are 6 apart and
-     * are offset at the start by the original stride: {@code [0, 6, 12, ...)},
-     * {@code [2, 8, 14, ...)}, and {@code [4, 10, 16, ...)}.
-     */
-    @Override
-    public List<? extends UnboundedSource<Long, CountingSource.CounterMark>> generateInitialSplits(
-        int desiredNumSplits, PipelineOptions options) throws Exception {
-      // Using Javadoc example, stride 2 with 3 splits becomes stride 6.
-      long newStride = stride * desiredNumSplits;
-
-      ImmutableList.Builder<UnboundedCountingSource> splits = ImmutableList.builder();
-      for (int i = 0; i < desiredNumSplits; ++i) {
-        // Starts offset by the original stride. Using Javadoc example, this generates starts of
-        // 0, 2, and 4.
-        splits.add(
-            new UnboundedCountingSource(
-                start + i * stride, newStride, elementsPerPeriod, period, timestampFn));
-      }
-      return splits.build();
-    }
-
-    @Override
-    public UnboundedReader<Long> createReader(
-        PipelineOptions options, CounterMark checkpointMark) {
-      return new UnboundedCountingReader(this, checkpointMark);
-    }
-
-    @Override
-    public Coder<CountingSource.CounterMark> getCheckpointMarkCoder() {
-      return AvroCoder.of(CountingSource.CounterMark.class);
-    }
-
-    @Override
-    public void validate() {}
-
-    @Override
-    public Coder<Long> getDefaultOutputCoder() {
-      return VarLongCoder.of();
-    }
-  }
-
-  /**
-   * The reader associated with {@link UnboundedCountingSource}.
-   *
-   * @see UnboundedCountingSource
-   */
-  private static class UnboundedCountingReader extends UnboundedReader<Long> {
-    private UnboundedCountingSource source;
-    private long current;
-    private Instant currentTimestamp;
-    private Instant firstStarted;
-
-    public UnboundedCountingReader(UnboundedCountingSource source, CounterMark mark) {
-      this.source = source;
-      if (mark == null) {
-        // Because we have not emitted an element yet, and start() calls advance, we need to
-        // "un-advance" so that start() produces the correct output.
-        this.current = source.start - source.stride;
-      } else {
-        this.current = mark.getLastEmitted();
-        this.firstStarted = mark.getStartTime();
-      }
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      if (firstStarted == null) {
-        this.firstStarted = Instant.now();
-      }
-      return advance();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      // Overflow-safe check that (current + source.stride) <= LONG.MAX_VALUE. Else, stop producing.
-      if (Long.MAX_VALUE - source.stride < current) {
-        return false;
-      }
-      long nextValue = current + source.stride;
-      if (expectedValue() < nextValue) {
-        return false;
-      }
-      current = nextValue;
-      currentTimestamp = source.timestampFn.apply(current);
-      return true;
-    }
-
-    private long expectedValue() {
-      if (source.period.getMillis() == 0L) {
-        return Long.MAX_VALUE;
-      }
-      double periodsElapsed =
-          (Instant.now().getMillis() - firstStarted.getMillis())
-              / (double) source.period.getMillis();
-      return (long) (source.elementsPerPeriod * periodsElapsed);
-    }
-
-    @Override
-    public Instant getWatermark() {
-      return source.timestampFn.apply(current);
-    }
-
-    @Override
-    public CounterMark getCheckpointMark() {
-      return new CounterMark(current, firstStarted);
-    }
-
-    @Override
-    public UnboundedSource<Long, CounterMark> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public Long getCurrent() throws NoSuchElementException {
-      return current;
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() throws NoSuchElementException {
-      return currentTimestamp;
-    }
-
-    @Override
-    public void close() throws IOException {}
-
-    @Override
-    public long getSplitBacklogBytes() {
-      long expected = expectedValue();
-      return Math.max(0L, 8 * (expected - current) / source.stride);
-    }
-  }
-
-  /**
-   * The checkpoint for an unbounded {@link CountingSource} is simply the last value produced. The
-   * associated source object encapsulates the information needed to produce the next value.
-   */
-  @DefaultCoder(AvroCoder.class)
-  public static class CounterMark implements UnboundedSource.CheckpointMark {
-    /** The last value emitted. */
-    private final long lastEmitted;
-    private final Instant startTime;
-
-    /**
-     * Creates a checkpoint mark reflecting the last emitted value.
-     */
-    public CounterMark(long lastEmitted, Instant startTime) {
-      this.lastEmitted = lastEmitted;
-      this.startTime = startTime;
-    }
-
-    /**
-     * Returns the last value emitted by the reader.
-     */
-    public long getLastEmitted() {
-      return lastEmitted;
-    }
-
-    /**
-     * Returns the time the reader was started.
-     */
-    public Instant getStartTime() {
-      return startTime;
-    }
-
-    /////////////////////////////////////////////////////////////////////////////////////
-
-    @SuppressWarnings("unused") // For AvroCoder
-    private CounterMark() {
-      this.lastEmitted = 0L;
-      this.startTime = Instant.now();
-    }
-
-    @Override
-    public void finalizeCheckpoint() throws IOException {}
-   }
-}


[54/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
index 3bbe5a5..f15f48e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+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.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
index 323b238..aee2427 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
index a2ded26..042b864 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
@@ -15,31 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES;
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES2;
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CollectionCoder;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.SetCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.LINES;
+import static org.apache.beam.sdk.TestUtils.LINES2;
+import static org.apache.beam.sdk.TestUtils.LINES_ARRAY;
+import static org.apache.beam.sdk.TestUtils.NO_LINES;
+import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CollectionCoder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.SetCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+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.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+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.PCollectionView;
+
 import com.google.common.collect.ImmutableSet;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
index 59729a0..1a7b0b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
@@ -15,36 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.KvMatcher.isKv;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.KvMatcher.isKv;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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.TimestampedValue;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java
index 91c7e65..dd01919 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
 
-import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
 import static org.hamcrest.Matchers.both;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@@ -28,8 +29,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.TestPipeline;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
index 27f894a..cf30940 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
index b084dbc..1a2d7f6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
index 7fb13a2..5446512 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java
index f70431c..5722365 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static org.junit.Assert.assertEquals;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java
index d3ba0fa..65c876e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java
@@ -15,15 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.transforms.Mean.CountSum;
-import com.google.cloud.dataflow.sdk.transforms.Mean.CountSumCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.Mean.CountSum;
+import org.apache.beam.sdk.transforms.Mean.CountSumCoder;
+
 import com.google.common.collect.Lists;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java
index 15e4210..8f1d301 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static org.junit.Assert.assertEquals;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpDoFn.java
index 428bfe3..a389fac 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpDoFn.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpDoFn.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 aa64f8f..0ef13c5 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
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.empty;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 4a55bea..5724dd6 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
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;
+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.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey;
-import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray;
 import static org.hamcrest.Matchers.isA;
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
@@ -31,30 +32,31 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.ParDo.Bound;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.ParDo.Bound;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
index 53642f1..dba6c16 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.Pipeline;
+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.Partition.PartitionFn;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java
index 2ae4a1f..0e919d2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 a2be424..0c2af3f 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
@@ -15,20 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+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 com.google.cloud.dataflow.sdk.TestUtils.LINES;
-import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.PCollection;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
index 81690ad..dc8355b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java
index 84189fb..4e98055 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn;
 import static org.junit.Assert.assertEquals;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
index a9eb7c2..1815cc9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
index 69656d6..238ba7b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 59919e4..3d4dfe0 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.assertEquals;
@@ -23,33 +23,34 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CustomCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.NullableCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+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.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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.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;
 
@@ -519,7 +520,7 @@ public class ViewTest implements Serializable {
 
     @Override
     public void verifyDeterministic()
-        throws com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException {
+        throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
       throw new NonDeterministicException(this, "Test coder is not deterministic on purpose.");
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
index ca01a72..6873624 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
index 6ccc85e..aee6a99 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.isA;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
index 7a06ab6..8cfb5c2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.display;
+package org.apache.beam.sdk.transforms.display;
 
 import static org.hamcrest.Matchers.allOf;
 
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
+import org.apache.beam.sdk.transforms.display.DisplayData.Item;
 
 import com.google.common.collect.Sets;
+
 import org.hamcrest.CustomTypeSafeMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.FeatureMatcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f24288e..5165e1a 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
@@ -15,22 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.display;
-
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes;
+package org.apache.beam.sdk.transforms.display;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace;
+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.includes;
+
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.core.StringStartsWith.startsWith;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 5e102e0..5aee8dd 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.display;
-
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue;
-import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes;
+package org.apache.beam.sdk.transforms.display;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace;
+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.includes;
+
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.everyItem;
@@ -38,14 +39,16 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.display.DisplayData.Item;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.testing.EqualsTester;
+
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java
index 123fecc..0a0a3f1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
 import static org.junit.Assert.assertFalse;
 
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.DoubleCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult.CoGbkResultCoder;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.join.CoGbkResult.CoGbkResultCoder;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java
index 27118d1..aa7cb5c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.emptyIterable;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.util.common.Reiterable;
-import com.google.cloud.dataflow.sdk.util.common.Reiterator;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import org.apache.beam.sdk.util.common.Reiterable;
+import org.apache.beam.sdk.util.common.Reiterator;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
index e9bf3a6..609f454 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
@@ -15,31 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+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.TupleTag;
+
 import com.google.common.collect.Iterables;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java
index 5668b7e..fb80fb6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.join;
+package org.apache.beam.sdk.transforms.join;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.DoubleCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Serializer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.Serializer;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
index 0499448..9ab4e5c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
index 5dde528..d3d43c9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
index 2483107..d70c4d6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
index 81ad601..eeda9ed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 04f37a1..4245e78 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
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
index 047f0ca..a44be90 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 7417953..00e2d22 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
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
index 6118a59..31e9a94 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
+
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn;
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set;
 import static org.junit.Assert.assertEquals;
 
 import org.joda.time.DateTime;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
index e9f87c3..b31ad56 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
index 9266b24..c8ee9ac 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
+
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn;
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -26,7 +27,7 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils;
+import org.apache.beam.sdk.testing.WindowFnTestUtils;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java
index 31ceadd..aaface2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+
 import com.google.common.collect.Lists;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
index 22dde66..e4e9cb3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger;
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java
index fe3d17c..ea4928e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertSame;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
index c98090e..9b74767 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
@@ -25,8 +25,8 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import com.google.cloud.dataflow.sdk.util.TriggerTester;
-import com.google.cloud.dataflow.sdk.util.TriggerTester.SimpleTriggerTester;
+import org.apache.beam.sdk.util.TriggerTester;
+import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java
index cff8a91..932d416 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn;
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set;
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.containsString;
@@ -26,7 +26,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils;
+import org.apache.beam.sdk.testing.WindowFnTestUtils;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
index a1776c8..a310cb3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
+
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
+import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn;
-import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils;
+import org.apache.beam.sdk.testing.WindowFnTestUtils;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
index 8bc0826..a8287a3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms.windowing;
+package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;


[28/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java
deleted file mode 100644
index c6dca2d..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,1369 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInGlobalWindow;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.startsWith;
-import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.AvroSource;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsList;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMap;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMultimap;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.TransformedMap;
-import com.google.cloud.dataflow.sdk.runners.dataflow.TestCountingSource;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import org.hamcrest.Description;
-import org.hamcrest.Matchers;
-import org.hamcrest.TypeSafeMatcher;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests for DataflowPipelineRunner.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineRunnerTest {
-
-  private static final String PROJECT_ID = "some-project";
-
-  @Rule
-  public TemporaryFolder tmpFolder = new TemporaryFolder();
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  // Asserts that the given Job has all expected fields set.
-  private static void assertValidJob(Job job) {
-    assertNull(job.getId());
-    assertNull(job.getCurrentState());
-  }
-
-  private DataflowPipeline buildDataflowPipeline(DataflowPipelineOptions options) {
-    options.setStableUniqueNames(CheckEnabled.ERROR);
-    DataflowPipeline p = DataflowPipeline.create(options);
-
-    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
-
-    return p;
-  }
-
-  private static Dataflow buildMockDataflow(
-      final ArgumentCaptor<Job> jobCaptor) throws IOException {
-    Dataflow mockDataflowClient = mock(Dataflow.class);
-    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
-    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
-    Dataflow.Projects.Jobs.Create mockRequest =
-        mock(Dataflow.Projects.Jobs.Create.class);
-    Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
-
-    when(mockDataflowClient.projects()).thenReturn(mockProjects);
-    when(mockProjects.jobs()).thenReturn(mockJobs);
-    when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
-        .thenReturn(mockRequest);
-    when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
-    when(mockList.setPageToken(anyString())).thenReturn(mockList);
-    when(mockList.execute())
-        .thenReturn(new ListJobsResponse().setJobs(
-            Arrays.asList(new Job()
-                              .setName("oldJobName")
-                              .setId("oldJobId")
-                              .setCurrentState("JOB_STATE_RUNNING"))));
-
-    Job resultJob = new Job();
-    resultJob.setId("newid");
-    when(mockRequest.execute()).thenReturn(resultJob);
-    return mockDataflowClient;
-  }
-
-  private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
-    GcsUtil mockGcsUtil = mock(GcsUtil.class);
-    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
-        .then(new Answer<SeekableByteChannel>() {
-              @Override
-              public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
-                return FileChannel.open(
-                    Files.createTempFile("channel-", ".tmp"),
-                    StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
-              }
-            });
-
-    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
-    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
-      @Override
-      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-      }
-    });
-    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
-    return mockGcsUtil;
-  }
-
-  private DataflowPipelineOptions buildPipelineOptions() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-    return buildPipelineOptions(jobCaptor);
-  }
-
-  private DataflowPipelineOptions buildPipelineOptions(
-      ArgumentCaptor<Job> jobCaptor) throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setProject(PROJECT_ID);
-    options.setTempLocation("gs://somebucket/some/path");
-    // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
-    options.setFilesToStage(new LinkedList<String>());
-    options.setDataflowClient(buildMockDataflow(jobCaptor));
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-    return options;
-  }
-
-  @Test
-  public void testRun() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    DataflowPipeline p = buildDataflowPipeline(options);
-    DataflowPipelineJob job = p.run();
-    assertEquals("newid", job.getJobId());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testRunReturnDifferentRequestId() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    Dataflow mockDataflowClient = options.getDataflowClient();
-    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
-    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
-        .thenReturn(mockRequest);
-    Job resultJob = new Job();
-    resultJob.setId("newid");
-    // Return a different request id.
-    resultJob.setClientRequestId("different_request_id");
-    when(mockRequest.execute()).thenReturn(resultJob);
-
-    DataflowPipeline p = buildDataflowPipeline(options);
-    try {
-      p.run();
-      fail("Expected DataflowJobAlreadyExistsException");
-    } catch (DataflowJobAlreadyExistsException expected) {
-      assertThat(expected.getMessage(),
-          containsString("If you want to submit a second job, try again by setting a "
-            + "different name using --jobName."));
-      assertEquals(expected.getJob().getJobId(), resultJob.getId());
-    }
-  }
-
-  @Test
-  public void testUpdate() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setUpdate(true);
-    options.setJobName("oldJobName");
-    DataflowPipeline p = buildDataflowPipeline(options);
-    DataflowPipelineJob job = p.run();
-    assertEquals("newid", job.getJobId());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testUpdateNonExistentPipeline() throws IOException {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Could not find running job named badJobName");
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setUpdate(true);
-    options.setJobName("badJobName");
-    DataflowPipeline p = buildDataflowPipeline(options);
-    p.run();
-  }
-
-  @Test
-  public void testUpdateAlreadyUpdatedPipeline() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setUpdate(true);
-    options.setJobName("oldJobName");
-    Dataflow mockDataflowClient = options.getDataflowClient();
-    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
-    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
-        .thenReturn(mockRequest);
-    final Job resultJob = new Job();
-    resultJob.setId("newid");
-    // Return a different request id.
-    resultJob.setClientRequestId("different_request_id");
-    when(mockRequest.execute()).thenReturn(resultJob);
-
-    DataflowPipeline p = buildDataflowPipeline(options);
-
-    thrown.expect(DataflowJobAlreadyUpdatedException.class);
-    thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
-      @Override
-      public void describeTo(Description description) {
-        description.appendText("Expected job ID: " + resultJob.getId());
-      }
-
-      @Override
-      protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
-        return resultJob.getId().equals(item.getJob().getJobId());
-      }
-    });
-    thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
-        + "into job id: newid and cannot be updated again.");
-    p.run();
-  }
-
-  @Test
-  public void testRunWithFiles() throws IOException {
-    // Test that the function DataflowPipelineRunner.stageFiles works as
-    // expected.
-    GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
-    final String gcsStaging = "gs://somebucket/some/path";
-    final String gcsTemp = "gs://somebucket/some/temp/path";
-    final String cloudDataflowDataset = "somedataset";
-
-    // Create some temporary files.
-    File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt");
-    temp1.deleteOnExit();
-    File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt");
-    temp2.deleteOnExit();
-
-    String overridePackageName = "alias.txt";
-
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setFilesToStage(ImmutableList.of(
-        temp1.getAbsolutePath(),
-        overridePackageName + "=" + temp2.getAbsolutePath()));
-    options.setStagingLocation(gcsStaging);
-    options.setTempLocation(gcsTemp);
-    options.setTempDatasetId(cloudDataflowDataset);
-    options.setProject(PROJECT_ID);
-    options.setJobName("job");
-    options.setDataflowClient(buildMockDataflow(jobCaptor));
-    options.setGcsUtil(mockGcsUtil);
-    options.setGcpCredential(new TestCredential());
-
-    DataflowPipeline p = buildDataflowPipeline(options);
-
-    DataflowPipelineJob job = p.run();
-    assertEquals("newid", job.getJobId());
-
-    Job workflowJob = jobCaptor.getValue();
-    assertValidJob(workflowJob);
-
-    assertEquals(
-        2,
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
-    DataflowPackage workflowPackage1 =
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
-    assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
-    DataflowPackage workflowPackage2 =
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
-    assertEquals(overridePackageName, workflowPackage2.getName());
-
-    assertEquals(
-        "storage.googleapis.com/somebucket/some/temp/path",
-        workflowJob.getEnvironment().getTempStoragePrefix());
-    assertEquals(
-        cloudDataflowDataset,
-        workflowJob.getEnvironment().getDataset());
-    assertEquals(
-        DataflowReleaseInfo.getReleaseInfo().getName(),
-        workflowJob.getEnvironment().getUserAgent().get("name"));
-    assertEquals(
-        DataflowReleaseInfo.getReleaseInfo().getVersion(),
-        workflowJob.getEnvironment().getUserAgent().get("version"));
-  }
-
-  @Test
-  public void runWithDefaultFilesToStage() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setFilesToStage(null);
-    DataflowPipelineRunner.fromOptions(options);
-    assertTrue(!options.getFilesToStage().isEmpty());
-  }
-
-  @Test
-  public void detectClassPathResourceWithFileResources() throws Exception {
-    File file = tmpFolder.newFile("file");
-    File file2 = tmpFolder.newFile("file2");
-    URLClassLoader classLoader = new URLClassLoader(new URL[]{
-        file.toURI().toURL(),
-        file2.toURI().toURL()
-    });
-
-    assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
-        DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader));
-  }
-
-  @Test
-  public void detectClassPathResourcesWithUnsupportedClassLoader() {
-    ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
-
-    DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader);
-  }
-
-  @Test
-  public void detectClassPathResourceWithNonFileResources() throws Exception {
-    String url = "http://www.google.com/all-the-secrets.jar";
-    URLClassLoader classLoader = new URLClassLoader(new URL[]{
-        new URL(url)
-    });
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Unable to convert url (" + url + ") to file.");
-
-    DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader);
-  }
-
-  @Test
-  public void testGcsStagingLocationInitialization() throws Exception {
-    // Test that the staging location is initialized correctly.
-    String gcsTemp = "gs://somebucket/some/temp/path";
-
-    // Set temp location (required), and check that staging location is set.
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setTempLocation(gcsTemp);
-    options.setProject(PROJECT_ID);
-    options.setGcpCredential(new TestCredential());
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    DataflowPipelineRunner.fromOptions(options);
-
-    assertNotNull(options.getStagingLocation());
-  }
-
-  @Test
-  public void testNonGcsFilePathInReadFailure() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
-
-    thrown.expectCause(Matchers.allOf(
-        instanceOf(IllegalArgumentException.class),
-        ThrowableMessageMatcher.hasMessage(
-            containsString("expected a valid 'gs://' path but was given"))));
-    p.run();
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNonGcsFilePathInWriteFailure() throws IOException {
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").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"));
-  }
-
-  @Test
-  public void testMultiSlashGcsFileReadPath() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadInvalidGcsFile")
-        .from("gs://bucket/tmp//file"));
-
-    thrown.expectCause(Matchers.allOf(
-        instanceOf(IllegalArgumentException.class),
-        ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
-    p.run();
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testMultiSlashGcsFileWritePath() throws IOException {
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("consecutive slashes");
-    pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
-  }
-
-  @Test
-  public void testInvalidTempLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setTempLocation("file://temp/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testInvalidStagingLocation() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setStagingLocation("file://my/staging/location");
-    try {
-      DataflowPipelineRunner.fromOptions(options);
-      fail("fromOptions should have failed");
-    } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
-    }
-    options.setStagingLocation("my/staging/location");
-    try {
-      DataflowPipelineRunner.fromOptions(options);
-      fail("fromOptions should have failed");
-    } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
-    }
-  }
-
-  @Test
-  public void testNonExistentTempLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setGcsUtil(mockGcsUtil);
-    options.setTempLocation("gs://non-existent-bucket/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString(
-        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNonExistentStagingLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setGcsUtil(mockGcsUtil);
-    options.setStagingLocation("gs://non-existent-bucket/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString(
-        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNoProjectFails() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-
-    options.setRunner(DataflowPipelineRunner.class);
-    // Explicitly set to null to prevent the default instance factory from reading credentials
-    // from a user's environment, causing this test to fail.
-    options.setProject(null);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project id");
-    thrown.expectMessage("when running a Dataflow in the cloud");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectId() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectPrefix() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("google.com:some-project-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectNumber() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project ID");
-    thrown.expectMessage("project number");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectDescription() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("some project");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project ID");
-    thrown.expectMessage("project description");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Number of worker harness threads");
-    thrown.expectMessage("Please make sure the value is non-negative.");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testNoStagingLocationAndNoTempLocationFails() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-project");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setGcpCredential(new TestCredential());
-    options.setProject("foo-project");
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setGcpCredential(new TestCredential());
-    options.setProject("foo-project");
-    options.setTempLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testInvalidJobName() throws IOException {
-    List<String> invalidNames = Arrays.asList(
-        "invalid_name",
-        "0invalid",
-        "invalid-");
-    List<String> expectedReason = Arrays.asList(
-        "JobName invalid",
-        "JobName invalid",
-        "JobName invalid");
-
-    for (int i = 0; i < invalidNames.size(); ++i) {
-      DataflowPipelineOptions options = buildPipelineOptions();
-      options.setJobName(invalidNames.get(i));
-
-      try {
-        DataflowPipelineRunner.fromOptions(options);
-        fail("Expected IllegalArgumentException for jobName "
-            + options.getJobName());
-      } catch (IllegalArgumentException e) {
-        assertThat(e.getMessage(),
-            containsString(expectedReason.get(i)));
-      }
-    }
-  }
-
-  @Test
-  public void testValidJobName() throws IOException {
-    List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
-        "this-one-is-fairly-long-01234567890123456789");
-
-    for (String name : names) {
-      DataflowPipelineOptions options = buildPipelineOptions();
-      options.setJobName(name);
-
-      DataflowPipelineRunner runner = DataflowPipelineRunner
-          .fromOptions(options);
-      assertNotNull(runner);
-    }
-  }
-
-  /**
-   * A fake PTransform for testing.
-   */
-  public static class TestTransform
-      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
-    public boolean translated = false;
-
-    @Override
-    public PCollection<Integer> apply(PCollection<Integer> input) {
-      return PCollection.<Integer>createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          input.isBounded());
-    }
-
-    @Override
-    protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
-      return input.getCoder();
-    }
-  }
-
-  @Test
-  public void testTransformTranslatorMissing() throws IOException {
-    // Test that we throw if we don't provide a translation.
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    DataflowPipeline p = DataflowPipeline.create(options);
-
-    p.apply(Create.of(Arrays.asList(1, 2, 3)))
-     .apply(new TestTransform());
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(Matchers.containsString("no translator registered"));
-    DataflowPipelineTranslator.fromOptions(options)
-        .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testTransformTranslator() throws IOException {
-    // Test that we can provide a custom translation
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipeline p = DataflowPipeline.create(options);
-    TestTransform transform = new TestTransform();
-
-    p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
-        .apply(transform);
-
-    DataflowPipelineTranslator translator = DataflowPipelineRunner
-        .fromOptions(options).getTranslator();
-
-    DataflowPipelineTranslator.registerTransformTranslator(
-        TestTransform.class,
-        new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
-          @SuppressWarnings("unchecked")
-          @Override
-          public void translate(
-              TestTransform transform,
-              DataflowPipelineTranslator.TranslationContext context) {
-            transform.translated = true;
-
-            // Note: This is about the minimum needed to fake out a
-            // translation. This obviously isn't a real translation.
-            context.addStep(transform, "TestTranslate");
-            context.addOutput("output", context.getOutput(transform));
-          }
-        });
-
-    translator.translate(
-        p, p.getRunner(), Collections.<DataflowPackage>emptyList());
-    assertTrue(transform.translated);
-  }
-
-  /** Records all the composite transforms visited within the Pipeline. */
-  private static class CompositeTransformRecorder implements PipelineVisitor {
-    private List<PTransform<?, ?>> transforms = new ArrayList<>();
-
-    @Override
-    public void enterCompositeTransform(TransformTreeNode node) {
-      if (node.getTransform() != null) {
-        transforms.add(node.getTransform());
-      }
-    }
-
-    @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void visitTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
-    }
-
-    public List<PTransform<?, ?>> getCompositeTransforms() {
-      return transforms;
-    }
-  }
-
-  @Test
-  public void testApplyIsScopedToExactClass() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    DataflowPipeline p = DataflowPipeline.create(options);
-
-    Create.TimestampedValues<String> transform =
-        Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
-    p.apply(transform);
-
-    CompositeTransformRecorder recorder = new CompositeTransformRecorder();
-    p.traverseTopologically(recorder);
-
-    assertThat("Expected to have seen CreateTimestamped composite transform.",
-        recorder.getCompositeTransforms(),
-        Matchers.<PTransform<?, ?>>contains(transform));
-  }
-
-  @Test
-  public void testToString() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    assertEquals("DataflowPipelineRunner#TestJobName",
-        DataflowPipelineRunner.fromOptions(options).toString());
-  }
-
-  private static PipelineOptions makeOptions(boolean streaming) {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setStreaming(streaming);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    return options;
-  }
-
-  private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
-      throws Exception {
-    String mode = streaming ? "streaming" : "batch";
-    thrown.expect(UnsupportedOperationException.class);
-    thrown.expectMessage(
-        "The DataflowPipelineRunner in " + mode + " mode does not support " + name);
-
-    Pipeline p = Pipeline.create(makeOptions(streaming));
-    p.apply(source);
-    p.run();
-  }
-
-  @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);
-  }
-
-  private void testUnsupportedSink(
-      PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
-          throws Exception {
-    thrown.expect(UnsupportedOperationException.class);
-    thrown.expectMessage(
-        "The DataflowPipelineRunner in streaming mode does not support " + name);
-
-    Pipeline p = Pipeline.create(makeOptions(streaming));
-    p.apply(Create.of("foo")).apply(sink);
-    p.run();
-  }
-
-  @Test
-  public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
-    testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
-  }
-
-  @Test
-  public void testTextIOSinkUnsupportedInStreaming() throws Exception {
-    testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
-  }
-
-  @Test
-  public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
-    DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
-        DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(ImmutableList.of("a", "b", "c")), contains(
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
-  }
-
-  @Test
-  public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(
-            new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
-                IntervalWindow.getCoder()));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(
-                    windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
-                )),
-            KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(
-                    windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
-                )));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
-        IsmRecord.of(ImmutableList.of(windowA, 0L),
-            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowA, 1L),
-            WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowA, 2L),
-            WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowB, 0L),
-            WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowB, 1L),
-            WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowC, 0L),
-            WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
-  }
-
-  @Test
-  public void testToIsmRecordForMapLikeDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
-            outputForSizeTag,
-            outputForEntrySetTag,
-            windowCoder,
-            keyCoder,
-            ismCoder,
-            false /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                // same window same key as to previous
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
-                // same window different key as to previous
-                KV.of(KV.of(2L, windowA),
-                    WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                // different window same key as to previous
-                KV.of(KV.of(2L, windowB),
-                    WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
-                // different window and different key as to previous
-                KV.of(KV.of(3L, windowB),
-                    WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                // different shard
-                KV.of(KV.of(4L, windowC),
-                    WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
-        IsmRecord.of(
-            ImmutableList.of(1L, windowA, 0L),
-            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(1L, windowA, 1L),
-            WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(2L, windowA, 0L),
-            WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(2L, windowB, 0L),
-            WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(3L, windowB, 0L),
-            WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(4L, windowC, 0L),
-            WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
-
-    // Verify the number of unique keys per window.
-    assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
-            KV.of(windowC, 1L))
-        ));
-
-    // Verify the output for the unique keys.
-    assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 1L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 3L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
-            KV.of(windowC, 4L))
-        ));
-  }
-
-  @Test
-  public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
-            outputForSizeTag,
-            outputForEntrySetTag,
-            windowCoder,
-            keyCoder,
-            ismCoder,
-            true /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-
-    Iterable<KV<Integer,
-                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                // same window same key as to previous
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
-
-    try {
-      doFnTester.processBatch(inputElements);
-      fail("Expected UserCodeException");
-    } catch (UserCodeException e) {
-      assertTrue(e.getCause() instanceof IllegalStateException);
-      IllegalStateException rootCause = (IllegalStateException) e.getCause();
-      assertThat(rootCause.getMessage(), containsString("Unique keys are expected but found key"));
-    }
-  }
-
-  @Test
-  public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
-        new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
-            windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowA, 2L),
-                    KV.of(windowA, 3L),
-                    KV.of(windowB, 7L))),
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowC, 9L))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
-        ));
-  }
-
-  @Test
-  public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
-        new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
-            keyCoder, windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowA, 2L),
-                    // same window as previous
-                    KV.of(windowA, 3L),
-                    // different window as previous
-                    KV.of(windowB, 3L))),
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowC, 3L))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
-        ));
-  }
-
-  @Test
-  public void testToMapDoFn() throws Exception {
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
-                  IsmRecord<WindowedValue<TransformedMap<Long,
-                                                         WindowedValue<Long>,
-                                                         Long>>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowC, WindowedValue.of(
-                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    List<IsmRecord<WindowedValue<TransformedMap<Long,
-                                                WindowedValue<Long>,
-                                                Long>>>> output =
-                                                doFnTester.processBatch(inputElements);
-    assertEquals(3, output.size());
-    Map<Long, Long> outputMap;
-
-    outputMap = output.get(0).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
-
-    outputMap = output.get(1).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
-
-    outputMap = output.get(2).getValue().getValue();
-    assertEquals(1, outputMap.size());
-    assertEquals(ImmutableMap.of(4L, 41L), outputMap);
-  }
-
-  @Test
-  public void testToMultimapDoFn() throws Exception {
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
-                  IsmRecord<WindowedValue<TransformedMap<Long,
-                                                         Iterable<WindowedValue<Long>>,
-                                                         Iterable<Long>>>>> doFnTester =
-        DoFnTester.of(
-            new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowC, WindowedValue.of(
-                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    List<IsmRecord<WindowedValue<TransformedMap<Long,
-                                                Iterable<WindowedValue<Long>>,
-                                                Iterable<Long>>>>> output =
-                                                doFnTester.processBatch(inputElements);
-    assertEquals(3, output.size());
-    Map<Long, Iterable<Long>> outputMap;
-
-    outputMap = output.get(0).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
-    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
-
-    outputMap = output.get(1).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
-    assertThat(outputMap.get(3L), containsInAnyOrder(31L));
-
-    outputMap = output.get(2).getValue().getValue();
-    assertEquals(1, outputMap.size());
-    assertThat(outputMap.get(4L), containsInAnyOrder(41L));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTest.java
deleted file mode 100644
index ebf0c9f..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTest.java
+++ /dev/null
@@ -1,45 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static org.junit.Assert.assertEquals;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowPipeline}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineTest {
-  @Test
-  public void testToString() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    options.setProject("project-id");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    assertEquals("DataflowPipeline#TestJobName",
-        DataflowPipeline.create(options).toString());
-  }
-}


[11/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRange.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRange.java
deleted file mode 100644
index c92e7e9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRange.java
+++ /dev/null
@@ -1,377 +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 com.google.cloud.dataflow.sdk.io.range;
-
-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.Verify.verify;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-
-/**
- * A class representing a range of {@link ByteKey ByteKeys}.
- *
- * <p>Instances of {@link ByteKeyRange} are immutable.
- *
- * <p>A {@link ByteKeyRange} enforces the restriction that its start and end keys must form a valid,
- * non-empty range {@code [startKey, endKey)} that is inclusive of the start key and exclusive of
- * the end key.
- *
- * <p>When the end key is empty, it is treated as the largest possible key.
- *
- * <h3>Interpreting {@link ByteKey} in a {@link ByteKeyRange}</h3>
- *
- * <p>The primary role of {@link ByteKeyRange} is to provide functionality for
- * {@link #estimateFractionForKey(ByteKey)}, {@link #interpolateKey(double)}, and
- * {@link #split(int)}, which are used for Google Cloud Dataflow's
- * <a href="https://cloud.google.com/dataflow/service/dataflow-service-desc#AutoScaling">Autoscaling
- * and Dynamic Work Rebalancing</a> features.
- *
- * <p>{@link ByteKeyRange} implements these features by treating a {@link ByteKey}'s underlying
- * {@code byte[]} as the binary expansion of floating point numbers in the range {@code [0.0, 1.0]}.
- * For example, the keys {@code ByteKey.of(0x80)}, {@code ByteKey.of(0xc0)}, and
- * {@code ByteKey.of(0xe0)} are interpreted as {@code 0.5}, {@code 0.75}, and {@code 0.875}
- * respectively. The empty {@code ByteKey.EMPTY} is interpreted as {@code 0.0} when used as the
- * start of a range and {@code 1.0} when used as the end key.
- *
- * <p>Key interpolation, fraction estimation, and range splitting are all interpreted in these
- * floating-point semantics. See the respective implementations for further details. <b>Note:</b>
- * the underlying implementations of these functions use {@link BigInteger} and {@link BigDecimal},
- * so they can be slow and should not be called in hot loops. Dataflow's dynamic work
- * rebalancing will only invoke these functions during periodic control operations, so they are not
- * called on the critical path.
- *
- * @see ByteKey
- */
-public final class ByteKeyRange implements Serializable {
-  private static final Logger logger = LoggerFactory.getLogger(ByteKeyRange.class);
-
-  /** The range of all keys, with empty start and end keys. */
-  public static final ByteKeyRange ALL_KEYS = ByteKeyRange.of(ByteKey.EMPTY, ByteKey.EMPTY);
-
-  /**
-   * Creates a new {@link ByteKeyRange} with the given start and end keys.
-   *
-   * <p>Note that if {@code endKey} is empty, it is treated as the largest possible key.
-   *
-   * @see ByteKeyRange
-   *
-   * @throws IllegalArgumentException if {@code endKey} is less than or equal to {@code startKey},
-   *     unless {@code endKey} is empty indicating the maximum possible {@link ByteKey}.
-   */
-  public static ByteKeyRange of(ByteKey startKey, ByteKey endKey) {
-    return new ByteKeyRange(startKey, endKey);
-  }
-
-  /**
-   * Returns the {@link ByteKey} representing the lower bound of this {@link ByteKeyRange}.
-   */
-  public ByteKey getStartKey() {
-    return startKey;
-  }
-
-  /**
-   * Returns the {@link ByteKey} representing the upper bound of this {@link ByteKeyRange}.
-   *
-   * <p>Note that if {@code endKey} is empty, it is treated as the largest possible key.
-   */
-  public ByteKey getEndKey() {
-    return endKey;
-  }
-
-  /**
-   * Returns {@code true} if the specified {@link ByteKey} is contained within this range.
-   */
-  public Boolean containsKey(ByteKey key) {
-    return key.compareTo(startKey) >= 0 && endsAfterKey(key);
-  }
-
-  /**
-   * Returns {@code true} if the specified {@link ByteKeyRange} overlaps this range.
-   */
-  public Boolean overlaps(ByteKeyRange other) {
-    // If each range starts before the other range ends, then they must overlap.
-    //     { [] } -- one range inside the other   OR   { [ } ] -- partial overlap.
-    return endsAfterKey(other.startKey) && other.endsAfterKey(startKey);
-  }
-
-  /**
-   * Returns a list of up to {@code numSplits + 1} {@link ByteKey ByteKeys} in ascending order,
-   * where the keys have been interpolated to form roughly equal sub-ranges of this
-   * {@link ByteKeyRange}, assuming a uniform distribution of keys within this range.
-   *
-   * <p>The first {@link ByteKey} in the result is guaranteed to be equal to {@link #getStartKey},
-   * and the last {@link ByteKey} in the result is guaranteed to be equal to {@link #getEndKey}.
-   * Thus the resulting list exactly spans the same key range as this {@link ByteKeyRange}.
-   *
-   * <p>Note that the number of keys returned is not always equal to {@code numSplits + 1}.
-   * Specifically, if this range is unsplittable (e.g., because the start and end keys are equal
-   * up to padding by zero bytes), the list returned will only contain the start and end key.
-   *
-   * @throws IllegalArgumentException if the specified number of splits is < 1
-   * @see ByteKeyRange the ByteKeyRange class Javadoc for more information about split semantics.
-   */
-  public List<ByteKey> split(int numSplits) {
-    checkArgument(numSplits > 0, "numSplits %s must be a positive integer", numSplits);
-
-    try {
-      ImmutableList.Builder<ByteKey> ret = ImmutableList.builder();
-      ret.add(startKey);
-      for (int i = 1; i < numSplits; ++i) {
-        ret.add(interpolateKey(i / (double) numSplits));
-      }
-      ret.add(endKey);
-      return ret.build();
-    } catch (IllegalStateException e) {
-      // The range is not splittable -- just return
-      return ImmutableList.of(startKey, endKey);
-    }
-  }
-
-  /**
-   * Returns the fraction of this range {@code [startKey, endKey)} that is in the interval
-   * {@code [startKey, key)}.
-   *
-   * @throws IllegalArgumentException if {@code key} does not fall within this range
-   * @see ByteKeyRange the ByteKeyRange class Javadoc for more information about fraction semantics.
-   */
-  public double estimateFractionForKey(ByteKey key) {
-    checkNotNull(key, "key");
-    checkArgument(!key.isEmpty(), "Cannot compute fraction for an empty key");
-    checkArgument(
-        key.compareTo(startKey) >= 0, "Expected key %s >= range start key %s", key, startKey);
-
-    if (key.equals(endKey)) {
-      return 1.0;
-    }
-    checkArgument(containsKey(key), "Cannot compute fraction for %s outside this %s", key, this);
-
-    byte[] startBytes = startKey.getBytes();
-    byte[] endBytes = endKey.getBytes();
-    byte[] keyBytes = key.getBytes();
-    // If the endKey is unspecified, add a leading 1 byte to it and a leading 0 byte to all other
-    // keys, to get a concrete least upper bound for the desired range.
-    if (endKey.isEmpty()) {
-      startBytes = addHeadByte(startBytes, (byte) 0);
-      endBytes = addHeadByte(endBytes, (byte) 1);
-      keyBytes = addHeadByte(keyBytes, (byte) 0);
-    }
-
-    // Pad to the longest of all 3 keys.
-    int paddedKeyLength = Math.max(Math.max(startBytes.length, endBytes.length), keyBytes.length);
-    BigInteger rangeStartInt = paddedPositiveInt(startBytes, paddedKeyLength);
-    BigInteger rangeEndInt = paddedPositiveInt(endBytes, paddedKeyLength);
-    BigInteger keyInt = paddedPositiveInt(keyBytes, paddedKeyLength);
-
-    // Keys are equal subject to padding by 0.
-    BigInteger range = rangeEndInt.subtract(rangeStartInt);
-    if (range.equals(BigInteger.ZERO)) {
-      logger.warn(
-          "Using 0.0 as the default fraction for this near-empty range {} where start and end keys"
-              + " differ only by trailing zeros.",
-          this);
-      return 0.0;
-    }
-
-    // Compute the progress (key-start)/(end-start) scaling by 2^64, dividing (which rounds),
-    // and then scaling down after the division. This gives ample precision when converted to
-    // double.
-    BigInteger progressScaled = keyInt.subtract(rangeStartInt).shiftLeft(64);
-    return progressScaled.divide(range).doubleValue() / Math.pow(2, 64);
-  }
-
-  /**
-   * Returns a {@link ByteKey} {@code key} such that {@code [startKey, key)} represents
-   * approximately the specified fraction of the range {@code [startKey, endKey)}. The interpolation
-   * is computed assuming a uniform distribution of keys.
-   *
-   * <p>For example, given the largest possible range (defined by empty start and end keys), the
-   * fraction {@code 0.5} will return the {@code ByteKey.of(0x80)}, which will also be returned for
-   * ranges {@code [0x40, 0xc0)} and {@code [0x6f, 0x91)}.
-   *
-   * <p>The key returned will never be empty.
-   *
-   * @throws IllegalArgumentException if {@code fraction} is outside the range [0, 1)
-   * @throws IllegalStateException if this range cannot be interpolated
-   * @see ByteKeyRange the ByteKeyRange class Javadoc for more information about fraction semantics.
-   */
-  public ByteKey interpolateKey(double fraction) {
-    checkArgument(
-        fraction >= 0.0 && fraction < 1.0, "Fraction %s must be in the range [0, 1)", fraction);
-    byte[] startBytes = startKey.getBytes();
-    byte[] endBytes = endKey.getBytes();
-    // If the endKey is unspecified, add a leading 1 byte to it and a leading 0 byte to all other
-    // keys, to get a concrete least upper bound for the desired range.
-    if (endKey.isEmpty()) {
-      startBytes = addHeadByte(startBytes, (byte) 0);
-      endBytes = addHeadByte(endBytes, (byte) 1);
-    }
-
-    // Pad to the longest key.
-    int paddedKeyLength = Math.max(startBytes.length, endBytes.length);
-    BigInteger rangeStartInt = paddedPositiveInt(startBytes, paddedKeyLength);
-    BigInteger rangeEndInt = paddedPositiveInt(endBytes, paddedKeyLength);
-
-    // If the keys are equal subject to padding by 0, we can't interpolate.
-    BigInteger range = rangeEndInt.subtract(rangeStartInt);
-    checkState(
-        !range.equals(BigInteger.ZERO),
-        "Refusing to interpolate for near-empty %s where start and end keys differ only by trailing"
-            + " zero bytes.",
-        this);
-
-    // Add precision so that range is at least 53 (double mantissa length) bits long. This way, we
-    // can interpolate small ranges finely, e.g., split the range key 3 to key 4 into 1024 parts.
-    // We add precision to range by adding zero bytes to the end of the keys, aka shifting the
-    // underlying BigInteger left by a multiple of 8 bits.
-    int bytesNeeded = ((53 - range.bitLength()) + 7) / 8;
-    if (bytesNeeded > 0) {
-      range = range.shiftLeft(bytesNeeded * 8);
-      rangeStartInt = rangeStartInt.shiftLeft(bytesNeeded * 8);
-      paddedKeyLength += bytesNeeded;
-    }
-
-    BigInteger interpolatedOffset =
-        new BigDecimal(range).multiply(BigDecimal.valueOf(fraction)).toBigInteger();
-
-    int outputKeyLength = endKey.isEmpty() ? (paddedKeyLength - 1) : paddedKeyLength;
-    return ByteKey.copyFrom(
-        fixupHeadZeros(rangeStartInt.add(interpolatedOffset).toByteArray(), outputKeyLength));
-  }
-
-  /**
-   * Returns new {@link ByteKeyRange} like this one, but with the specified start key.
-   */
-  public ByteKeyRange withStartKey(ByteKey startKey) {
-    return new ByteKeyRange(startKey, endKey);
-  }
-
-  /**
-   * Returns new {@link ByteKeyRange} like this one, but with the specified end key.
-   */
-  public ByteKeyRange withEndKey(ByteKey endKey) {
-    return new ByteKeyRange(startKey, endKey);
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////
-  private final ByteKey startKey;
-  private final ByteKey endKey;
-
-  private ByteKeyRange(ByteKey startKey, ByteKey endKey) {
-    this.startKey = checkNotNull(startKey, "startKey");
-    this.endKey = checkNotNull(endKey, "endKey");
-    checkArgument(endsAfterKey(startKey), "Start %s must be less than end %s", startKey, endKey);
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(ByteKeyRange.class)
-        .add("startKey", startKey)
-        .add("endKey", endKey)
-        .toString();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o == this) {
-      return true;
-    }
-    if (!(o instanceof ByteKeyRange)) {
-      return false;
-    }
-    ByteKeyRange other = (ByteKeyRange) o;
-    return Objects.equals(startKey, other.startKey) && Objects.equals(endKey, other.endKey);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(startKey, endKey);
-  }
-
-  /**
-   * Returns a copy of the specified array with the specified byte added at the front.
-   */
-  private static byte[] addHeadByte(byte[] array, byte b) {
-    byte[] ret = new byte[array.length + 1];
-    ret[0] = b;
-    System.arraycopy(array, 0, ret, 1, array.length);
-    return ret;
-  }
-
-  /**
-   * Ensures the array is exactly {@code size} bytes long. Returns the input array if the condition
-   * is met, otherwise either adds or removes zero bytes from the beginning of {@code array}.
-   */
-  private static byte[] fixupHeadZeros(byte[] array, int size) {
-    int padding = size - array.length;
-    if (padding == 0) {
-      return array;
-    }
-
-    if (padding < 0) {
-      // There is one zero byte at the beginning, added by BigInteger to make there be a sign
-      // bit when converting to bytes.
-      verify(
-          padding == -1,
-          "key %s: expected length %d with exactly one byte of padding, found %d",
-          ByteKey.copyFrom(array),
-          size,
-          -padding);
-      verify(
-          (array[0] == 0) && ((array[1] & 0x80) == 0x80),
-          "key %s: is 1 byte longer than expected, indicating BigInteger padding. Expect first byte"
-              + " to be zero with set MSB in second byte.",
-          ByteKey.copyFrom(array));
-      return Arrays.copyOfRange(array, 1, array.length);
-    }
-
-    byte[] ret = new byte[size];
-    System.arraycopy(array, 0, ret, padding, array.length);
-    return ret;
-  }
-
-  /**
-   * Returns {@code true} when the specified {@code key} is smaller this range's end key. The only
-   * semantic change from {@code (key.compareTo(getEndKey()) < 0)} is that the empty end key is
-   * treated as larger than all possible {@link ByteKey keys}.
-   */
-  boolean endsAfterKey(ByteKey key) {
-    return endKey.isEmpty() || key.compareTo(endKey) < 0;
-  }
-
-  /** Builds a BigInteger out of the specified array, padded to the desired byte length. */
-  private static BigInteger paddedPositiveInt(byte[] bytes, int length) {
-    int bytePaddingNeeded = length - bytes.length;
-    checkArgument(
-        bytePaddingNeeded >= 0, "Required bytes.length {} < length {}", bytes.length, length);
-    BigInteger ret = new BigInteger(1, bytes);
-    return (bytePaddingNeeded == 0) ? ret : ret.shiftLeft(8 * bytePaddingNeeded);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRangeTracker.java
deleted file mode 100644
index 0d01f8f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/ByteKeyRangeTracker.java
+++ /dev/null
@@ -1,118 +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 com.google.cloud.dataflow.sdk.io.range;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link RangeTracker} for {@link ByteKey ByteKeys} in {@link ByteKeyRange ByteKeyRanges}.
- *
- * @see ByteKey
- * @see ByteKeyRange
- */
-public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
-  private static final Logger logger = LoggerFactory.getLogger(ByteKeyRangeTracker.class);
-
-  /** Instantiates a new {@link ByteKeyRangeTracker} with the specified range. */
-  public static ByteKeyRangeTracker of(ByteKeyRange range) {
-    return new ByteKeyRangeTracker(range);
-  }
-
-  @Override
-  public synchronized ByteKey getStartPosition() {
-    return range.getStartKey();
-  }
-
-  @Override
-  public synchronized ByteKey getStopPosition() {
-    return range.getEndKey();
-  }
-
-  @Override
-  public synchronized boolean tryReturnRecordAt(boolean isAtSplitPoint, ByteKey recordStart) {
-    if (isAtSplitPoint && !range.containsKey(recordStart)) {
-      return false;
-    }
-    position = recordStart;
-    return true;
-  }
-
-  @Override
-  public synchronized boolean trySplitAtPosition(ByteKey splitPosition) {
-    // Unstarted.
-    if (position == null) {
-      logger.warn(
-          "{}: Rejecting split request at {} because no records have been returned.",
-          this,
-          splitPosition);
-      return false;
-    }
-
-    // Started, but not after current position.
-    if (splitPosition.compareTo(position) <= 0) {
-      logger.warn(
-          "{}: Rejecting split request at {} because it is not after current position {}.",
-          this,
-          splitPosition,
-          position);
-      return false;
-    }
-
-    // Sanity check.
-    if (!range.containsKey(splitPosition)) {
-      logger.warn(
-          "{}: Rejecting split request at {} because it is not within the range.",
-          this,
-          splitPosition);
-      return false;
-    }
-
-    range = range.withEndKey(splitPosition);
-    return true;
-  }
-
-  @Override
-  public synchronized double getFractionConsumed() {
-    if (position == null) {
-      return 0;
-    }
-    return range.estimateFractionForKey(position);
-  }
-
-  ///////////////////////////////////////////////////////////////////////////////
-  private ByteKeyRange range;
-  @Nullable private ByteKey position;
-
-  private ByteKeyRangeTracker(ByteKeyRange range) {
-    this.range = range;
-    this.position = null;
-  }
-
-  @Override
-  public String toString() {
-    return toStringHelper(ByteKeyRangeTracker.class)
-        .add("range", range)
-        .add("position", position)
-        .toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/OffsetRangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/OffsetRangeTracker.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/OffsetRangeTracker.java
deleted file mode 100644
index 3cebb77..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/OffsetRangeTracker.java
+++ /dev/null
@@ -1,183 +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 com.google.cloud.dataflow.sdk.io.range;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link RangeTracker} for non-negative positions of type {@code long}.
- */
-public class OffsetRangeTracker implements RangeTracker<Long> {
-  private static final Logger LOG = LoggerFactory.getLogger(OffsetRangeTracker.class);
-
-  private final long startOffset;
-  private long stopOffset;
-  private long lastRecordStart = -1L;
-  private long offsetOfLastSplitPoint = -1L;
-
-  /**
-   * Offset corresponding to infinity. This can only be used as the upper-bound of a range, and
-   * indicates reading all of the records until the end without specifying exactly what the end is.
-   *
-   * <p>Infinite ranges cannot be split because it is impossible to estimate progress within them.
-   */
-  public static final long OFFSET_INFINITY = Long.MAX_VALUE;
-
-  /**
-   * Creates an {@code OffsetRangeTracker} for the specified range.
-   */
-  public OffsetRangeTracker(long startOffset, long stopOffset) {
-    this.startOffset = startOffset;
-    this.stopOffset = stopOffset;
-  }
-
-  @Override
-  public synchronized Long getStartPosition() {
-    return startOffset;
-  }
-
-  @Override
-  public synchronized Long getStopPosition() {
-    return stopOffset;
-  }
-
-  @Override
-  public boolean tryReturnRecordAt(boolean isAtSplitPoint, Long recordStart) {
-    return tryReturnRecordAt(isAtSplitPoint, recordStart.longValue());
-  }
-
-  public synchronized boolean tryReturnRecordAt(boolean isAtSplitPoint, long recordStart) {
-    if (lastRecordStart == -1 && !isAtSplitPoint) {
-      throw new IllegalStateException(
-          String.format("The first record [starting at %d] must be at a split point", recordStart));
-    }
-    if (recordStart < lastRecordStart) {
-      throw new IllegalStateException(
-          String.format(
-              "Trying to return record [starting at %d] "
-                  + "which is before the last-returned record [starting at %d]",
-              recordStart,
-              lastRecordStart));
-    }
-    if (isAtSplitPoint) {
-      if (offsetOfLastSplitPoint != -1L && recordStart == offsetOfLastSplitPoint) {
-        throw new IllegalStateException(
-            String.format(
-                "Record at a split point has same offset as the previous split point: "
-                    + "previous split point at %d, current record starts at %d",
-                offsetOfLastSplitPoint, recordStart));
-      }
-      if (recordStart >= stopOffset) {
-        return false;
-      }
-      offsetOfLastSplitPoint = recordStart;
-    }
-
-    lastRecordStart = recordStart;
-    return true;
-  }
-
-  @Override
-  public boolean trySplitAtPosition(Long splitOffset) {
-    return trySplitAtPosition(splitOffset.longValue());
-  }
-
-  public synchronized boolean trySplitAtPosition(long splitOffset) {
-    if (stopOffset == OFFSET_INFINITY) {
-      LOG.debug("Refusing to split {} at {}: stop position unspecified", this, splitOffset);
-      return false;
-    }
-    if (lastRecordStart == -1) {
-      LOG.debug("Refusing to split {} at {}: unstarted", this, splitOffset);
-      return false;
-    }
-
-    // Note: technically it is correct to split at any position after the last returned
-    // split point, not just the last returned record.
-    // TODO: Investigate whether in practice this is useful or, rather, confusing.
-    if (splitOffset <= lastRecordStart) {
-      LOG.debug(
-          "Refusing to split {} at {}: already past proposed split position", this, splitOffset);
-      return false;
-    }
-    if (splitOffset < startOffset || splitOffset >= stopOffset) {
-      LOG.debug(
-          "Refusing to split {} at {}: proposed split position out of range", this, splitOffset);
-      return false;
-    }
-    LOG.debug("Agreeing to split {} at {}", this, splitOffset);
-    this.stopOffset = splitOffset;
-    return true;
-  }
-
-  /**
-   * Returns a position {@code P} such that the range {@code [start, P)} represents approximately
-   * the given fraction of the range {@code [start, end)}. Assumes that the density of records
-   * in the range is approximately uniform.
-   */
-  public synchronized long getPositionForFractionConsumed(double fraction) {
-    if (stopOffset == OFFSET_INFINITY) {
-      throw new IllegalArgumentException(
-          "getPositionForFractionConsumed is not applicable to an unbounded range: " + this);
-    }
-    return (long) Math.ceil(startOffset + fraction * (stopOffset - startOffset));
-  }
-
-  @Override
-  public synchronized double getFractionConsumed() {
-    if (stopOffset == OFFSET_INFINITY) {
-      return 0.0;
-    }
-    if (lastRecordStart == -1) {
-      return 0.0;
-    }
-    // E.g., when reading [3, 6) and lastRecordStart is 4, that means we consumed 3,4 of 3,4,5
-    // which is (4 - 3 + 1) / (6 - 3) = 67%.
-    // Also, clamp to at most 1.0 because the last consumed position can extend past the
-    // stop position.
-    return Math.min(1.0, 1.0 * (lastRecordStart - startOffset + 1) / (stopOffset - startOffset));
-  }
-
-  @Override
-  public synchronized String toString() {
-    String stopString = (stopOffset == OFFSET_INFINITY) ? "infinity" : String.valueOf(stopOffset);
-    if (lastRecordStart >= 0) {
-      return String.format(
-          "<at [starting at %d] of offset range [%d, %s)>",
-          lastRecordStart,
-          startOffset,
-          stopString);
-    } else {
-      return String.format("<unstarted in offset range [%d, %s)>", startOffset, stopString);
-    }
-  }
-
-  /**
-   * Returns a copy of this tracker for testing purposes (to simplify testing methods with
-   * side effects).
-   */
-  @VisibleForTesting
-  OffsetRangeTracker copy() {
-    OffsetRangeTracker res = new OffsetRangeTracker(startOffset, stopOffset);
-    res.lastRecordStart = this.lastRecordStart;
-    return res;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/RangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/RangeTracker.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/RangeTracker.java
deleted file mode 100644
index dad9edc..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/RangeTracker.java
+++ /dev/null
@@ -1,221 +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 com.google.cloud.dataflow.sdk.io.range;
-
-/**
- * A {@code RangeTracker} is a thread-safe helper object for implementing dynamic work rebalancing
- * in position-based {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader}
- * subclasses.
- *
- * <h3>Usage of the RangeTracker class hierarchy</h3>
- * The abstract {@code RangeTracker} interface should not be used per se - all users should use its
- * subclasses directly. We declare it here because all subclasses have roughly the same interface
- * and the same properties, to centralize the documentation. Currently we provide one
- * implementation - {@link OffsetRangeTracker}.
- *
- * <h3>Position-based sources</h3>
- * A position-based source is one where the source can be described by a range of positions of
- * an ordered type and the records returned by the reader can be described by positions of the
- * same type.
- *
- * <p>In case a record occupies a range of positions in the source, the most important thing about
- * the record is the position where it starts.
- *
- * <p>Defining the semantics of positions for a source is entirely up to the source class, however
- * the chosen definitions have to obey certain properties in order to make it possible to correctly
- * split the source into parts, including dynamic splitting. Two main aspects need to be defined:
- * <ul>
- *   <li>How to assign starting positions to records.
- *   <li>Which records should be read by a source with a range {@code [A, B)}.
- * </ul>
- * Moreover, reading a range must be <i>efficient</i>, i.e., the performance of reading a range
- * should not significantly depend on the location of the range. For example, reading the range
- * {@code [A, B)} should not require reading all data before {@code A}.
- *
- * <p>The sections below explain exactly what properties these definitions must satisfy, and
- * how to use a {@code RangeTracker} with a properly defined source.
- *
- * <h3>Properties of position-based sources</h3>
- * The main requirement for position-based sources is <i>associativity</i>: reading records from
- * {@code [A, B)} and records from {@code [B, C)} should give the same records as reading from
- * {@code [A, C)}, where {@code A <= B <= C}. This property ensures that no matter how a range
- * of positions is split into arbitrarily many sub-ranges, the total set of records described by
- * them stays the same.
- *
- * <p>The other important property is how the source's range relates to positions of records in
- * the source. In many sources each record can be identified by a unique starting position.
- * In this case:
- * <ul>
- *   <li>All records returned by a source {@code [A, B)} must have starting positions
- *   in this range.
- *   <li>All but the last record should end within this range. The last record may or may not
- *   extend past the end of the range.
- *   <li>Records should not overlap.
- * </ul>
- * Such sources should define "read {@code [A, B)}" as "read from the first record starting at or
- * after A, up to but not including the first record starting at or after B".
- *
- * <p>Some examples of such sources include reading lines or CSV from a text file, reading keys and
- * values from a BigTable, etc.
- *
- * <p>The concept of <i>split points</i> allows to extend the definitions for dealing with sources
- * where some records cannot be identified by a unique starting position.
- *
- * <p>In all cases, all records returned by a source {@code [A, B)} must <i>start</i> at or after
- * {@code A}.
- *
- * <h3>Split points</h3>
- *
- * <p>Some sources may have records that are not directly addressable. For example, imagine a file
- * format consisting of a sequence of compressed blocks. Each block can be assigned an offset, but
- * records within the block cannot be directly addressed without decompressing the block. Let us
- * refer to this hypothetical format as <i>CBF (Compressed Blocks Format)</i>.
- *
- * <p>Many such formats can still satisfy the associativity property. For example, in CBF, reading
- * {@code [A, B)} can mean "read all the records in all blocks whose starting offset is in
- * {@code [A, B)}".
- *
- * <p>To support such complex formats, we introduce the notion of <i>split points</i>. We say that
- * a record is a split point if there exists a position {@code A} such that the record is the first
- * one to be returned when reading the range  {@code [A, infinity)}. In CBF, the only split points
- * would be the first records in each block.
- *
- * <p>Split points allow us to define the meaning of a record's position and a source's range
- * in all cases:
- * <ul>
- *   <li>For a record that is at a split point, its position is defined to be the largest
- *   {@code A} such that reading a source with the range {@code [A, infinity)} returns this record;
- *   <li>Positions of other records are only required to be non-decreasing;
- *   <li>Reading the source {@code [A, B)} must return records starting from the first split point
- *   at or after {@code A}, up to but not including the first split point at or after {@code B}.
- *   In particular, this means that the first record returned by a source MUST always be
- *   a split point.
- *   <li>Positions of split points must be unique.
- * </ul>
- * As a result, for any decomposition of the full range of the source into position ranges, the
- * total set of records will be the full set of records in the source, and each record
- * will be read exactly once.
- *
- * <h3>Consumed positions</h3>
- * As the source is being read, and records read from it are being passed to the downstream
- * transforms in the pipeline, we say that positions in the source are being <i>consumed</i>.
- * When a reader has read a record (or promised to a caller that a record will be returned),
- * positions up to and including the record's start position are considered <i>consumed</i>.
- *
- * <p>Dynamic splitting can happen only at <i>unconsumed</i> positions. If the reader just
- * returned a record at offset 42 in a file, dynamic splitting can happen only at offset 43 or
- * beyond, as otherwise that record could be read twice (by the current reader and by a reader
- * of the task starting at 43).
- *
- * <h3>Example</h3>
- * The following example uses an {@link OffsetRangeTracker} to support dynamically splitting
- * a source with integer positions (offsets).
- * <pre> {@code
- *   class MyReader implements BoundedReader<Foo> {
- *     private MySource currentSource;
- *     private final OffsetRangeTracker tracker = new OffsetRangeTracker();
- *     ...
- *     MyReader(MySource source) {
- *       this.currentSource = source;
- *       this.tracker = new MyRangeTracker<>(source.getStartOffset(), source.getEndOffset())
- *     }
- *     ...
- *     boolean start() {
- *       ... (general logic for locating the first record) ...
- *       if (!tracker.tryReturnRecordAt(true, recordStartOffset)) return false;
- *       ... (any logic that depends on the record being returned, e.g. counting returned records)
- *       return true;
- *     }
- *     boolean advance() {
- *       ... (general logic for locating the next record) ...
- *       if (!tracker.tryReturnRecordAt(isAtSplitPoint, recordStartOffset)) return false;
- *       ... (any logic that depends on the record being returned, e.g. counting returned records)
- *       return true;
- *     }
- *
- *     double getFractionConsumed() {
- *       return tracker.getFractionConsumed();
- *     }
- *   }
- * } </pre>
- *
- * <h3>Usage with different models of iteration</h3>
- * When using this class to protect a
- * {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader}, follow the pattern
- * described above.
- *
- * <p>When using this class to protect iteration in the {@code hasNext()/next()}
- * model, consider the record consumed when {@code hasNext()} is about to return true, rather than
- * when {@code next()} is called, because {@code hasNext()} returning true is promising the caller
- * that {@code next()} will have an element to return - so {@link #trySplitAtPosition} must not
- * split the range in a way that would make the record promised by {@code hasNext()} belong to
- * a different range.
- *
- * <p>Also note that implementations of {@code hasNext()} need to ensure
- * that they call {@link #tryReturnRecordAt} only once even if {@code hasNext()} is called
- * repeatedly, due to the requirement on uniqueness of split point positions.
- *
- * @param <PositionT> Type of positions used by the source to define ranges and identify records.
- */
-public interface RangeTracker<PositionT> {
-  /**
-   * Returns the starting position of the current range, inclusive.
-   */
-  PositionT getStartPosition();
-
-  /**
-   * Returns the ending position of the current range, exclusive.
-   */
-  PositionT getStopPosition();
-
-  /**
-   * Atomically determines whether a record at the given position can be returned and updates
-   * internal state. In particular:
-   * <ul>
-   *   <li>If {@code isAtSplitPoint} is {@code true}, and {@code recordStart} is outside the current
-   *   range, returns {@code false};
-   *   <li>Otherwise, updates the last-consumed position to {@code recordStart} and returns
-   *   {@code true}.
-   * </ul>
-   * <p>This method MUST be called on all split point records. It may be called on every record.
-   */
-  boolean tryReturnRecordAt(boolean isAtSplitPoint, PositionT recordStart);
-
-  /**
-   * Atomically splits the current range [{@link #getStartPosition}, {@link #getStopPosition})
-   * into a "primary" part [{@link #getStartPosition}, {@code splitPosition})
-   * and a "residual" part [{@code splitPosition}, {@link #getStopPosition}), assuming the current
-   * last-consumed position is within [{@link #getStartPosition}, splitPosition)
-   * (i.e., {@code splitPosition} has not been consumed yet).
-   *
-   * <p>Updates the current range to be the primary and returns {@code true}. This means that
-   * all further calls on the current object will interpret their arguments relative to the
-   * primary range.
-   *
-   * <p>If the split position has already been consumed, or if no {@link #tryReturnRecordAt} call
-   * was made yet, returns {@code false}. The second condition is to prevent dynamic splitting
-   * during reader start-up.
-   */
-  boolean trySplitAtPosition(PositionT splitPosition);
-
-  /**
-   * Returns the approximate fraction of positions in the source that have been consumed by
-   * successful {@link #tryReturnRecordAt} calls, or 0.0 if no such calls have happened.
-   */
-  double getFractionConsumed();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/package-info.java
deleted file mode 100644
index 9117e43..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/range/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-/**
- * Provides thread-safe helpers for implementing dynamic work rebalancing in position-based
- * bounded sources.
- *
- * <p>See {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} to get started.
- */
-package com.google.cloud.dataflow.sdk.io.range;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java
deleted file mode 100644
index 2860fbe..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * Options that allow setting the application name.
- */
-public interface ApplicationNameOptions extends PipelineOptions {
-  /**
-   * Name of application, for display purposes.
-   *
-   * <p>Defaults to the name of the class that constructs the {@link PipelineOptions}
-   * via the {@link PipelineOptionsFactory}.
-   */
-  @Description("Name of application for display purposes. Defaults to the name of the class that "
-      + "constructs the PipelineOptions via the PipelineOptionsFactory.")
-  String getAppName();
-  void setAppName(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java
deleted file mode 100644
index 5576b5f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java
+++ /dev/null
@@ -1,32 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * Properties needed when using BigQuery with the Dataflow SDK.
- */
-@Description("Options that are used to configure BigQuery. See "
-    + "https://cloud.google.com/bigquery/what-is-bigquery for details on BigQuery.")
-public interface BigQueryOptions extends ApplicationNameOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-  @Description("Temporary dataset for BigQuery table operations. "
-      + "Supported values are \"bigquery.googleapis.com/{dataset}\"")
-  @Default.String("bigquery.googleapis.com/cloud_dataflow")
-  String getTempDatasetId();
-  void setTempDatasetId(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java
deleted file mode 100644
index 843eba1..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java
+++ /dev/null
@@ -1,154 +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 com.google.cloud.dataflow.sdk.options;
-
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * {@link Default} represents a set of annotations that can be used to annotate getter properties
- * on {@link PipelineOptions} with information representing the default value to be returned
- * if no value is specified.
- */
-public @interface Default {
-  /**
-   * This represents that the default of the option is the specified {@link java.lang.Class} value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Class {
-    java.lang.Class<?> value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified {@link java.lang.String}
-   * value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface String {
-    java.lang.String value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified boolean primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Boolean {
-    boolean value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified char primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Character {
-    char value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified byte primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Byte {
-    byte value();
-  }
-  /**
-   * This represents that the default of the option is the specified short primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Short {
-    short value();
-  }
-  /**
-   * This represents that the default of the option is the specified int primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Integer {
-    int value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified long primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Long {
-    long value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified float primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Float {
-    float value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified double primitive value.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Double {
-    double value();
-  }
-
-  /**
-   * This represents that the default of the option is the specified enum.
-   * The value should equal the enum's {@link java.lang.Enum#name() name}.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Enum {
-    java.lang.String value();
-  }
-
-  /**
-   * Value must be of type {@link DefaultValueFactory} and have a default constructor.
-   * Value is instantiated and then used as a factory to generate the default.
-   *
-   * <p>See {@link DefaultValueFactory} for more details.
-   */
-  @Target(ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface InstanceFactory {
-    java.lang.Class<? extends DefaultValueFactory<?>> value();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java
deleted file mode 100644
index a6e7856..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java
+++ /dev/null
@@ -1,39 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * An interface used with the {@link Default.InstanceFactory} annotation to specify the class that
- * will be an instance factory to produce default values for a given getter on
- * {@link PipelineOptions}. When a property on a {@link PipelineOptions} is fetched, and is
- * currently unset, the default value factory will be instantiated and invoked.
- *
- * <p>Care must be taken to not produce an infinite loop when accessing other fields on the
- * {@link PipelineOptions} object.
- *
- * @param <T> The type of object this factory produces.
- */
-public interface DefaultValueFactory<T> {
-  /**
-   * Creates a default value for a getter marked with {@link Default.InstanceFactory}.
-   *
-   * @param options The current pipeline options.
-   * @return The default value to be used for the annotated getter.
-   */
-  T create(PipelineOptions options);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java
deleted file mode 100644
index 4f90c2a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java
+++ /dev/null
@@ -1,36 +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 com.google.cloud.dataflow.sdk.options;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * Descriptions are used to generate human readable output when the {@code --help}
- * command is specified. Description annotations placed on interfaces that extend
- * {@link PipelineOptions} will describe groups of related options. Description annotations
- * placed on getter methods will be used to provide human readable information
- * for the specific option.
- */
-@Target({ElementType.METHOD, ElementType.TYPE})
-@Retention(RetentionPolicy.RUNTIME)
-public @interface Description {
-  String value();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java
deleted file mode 100644
index a505632..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipeline;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-/**
- * Options that can be used to configure the {@link DirectPipeline}.
- */
-public interface DirectPipelineOptions extends
-    ApplicationNameOptions, BigQueryOptions, GcsOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-
-  /**
-   * The random seed to use for pseudorandom behaviors in the {@link DirectPipelineRunner}.
-   * If not explicitly specified, a random seed will be generated.
-   */
-  @JsonIgnore
-  @Description("The random seed to use for pseudorandom behaviors in the DirectPipelineRunner."
-      + " If not explicitly specified, a random seed will be generated.")
-  Long getDirectPipelineRunnerRandomSeed();
-  void setDirectPipelineRunnerRandomSeed(Long value);
-
-  /**
-   * Controls whether the runner should ensure that all of the elements of
-   * the pipeline, such as DoFns, can be serialized.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should ensure that all of the elements of the "
-      + "pipeline, such as DoFns, can be serialized.")
-  @Default.Boolean(true)
-  boolean isTestSerializability();
-  void setTestSerializability(boolean testSerializability);
-
-  /**
-   * Controls whether the runner should ensure that all of the elements of
-   * every {@link PCollection} can be encoded using the appropriate
-   * {@link Coder}.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should ensure that all of the elements of every "
-      + "PCollection can be encoded using the appropriate Coder.")
-  @Default.Boolean(true)
-  boolean isTestEncodability();
-  void setTestEncodability(boolean testEncodability);
-
-  /**
-   * Controls whether the runner should randomize the order of each
-   * {@link PCollection}.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should randomize the order of each PCollection.")
-  @Default.Boolean(true)
-  boolean isTestUnorderedness();
-  void setTestUnorderedness(boolean testUnorderedness);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
deleted file mode 100644
index 52028cd..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
+++ /dev/null
@@ -1,292 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants;
-import com.google.cloud.dataflow.sdk.util.CredentialFactory;
-import com.google.cloud.dataflow.sdk.util.GcpCredentialFactory;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.io.Files;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.security.GeneralSecurityException;
-import java.util.Locale;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * Options used to configure Google Cloud Platform project and credentials.
- *
- * <p>These options configure which of the following three different mechanisms for obtaining a
- * credential are used:
- * <ol>
- *   <li>
- *     It can fetch the
- *     <a href="https://developers.google.com/accounts/docs/application-default-credentials">
- *     application default credentials</a>.
- *   </li>
- *   <li>
- *     The user can specify a client secrets file and go through the OAuth2
- *     webflow. The credential will then be cached in the user's home
- *     directory for reuse.
- *   </li>
- *   <li>
- *     The user can specify a file containing a service account private key along
- *     with the service account name.
- *   </li>
- * </ol>
- *
- * <p>The default mechanism is to use the
- * <a href="https://developers.google.com/accounts/docs/application-default-credentials">
- * application default credentials</a>. The other options can be
- * used by setting the corresponding properties.
- */
-@Description("Options used to configure Google Cloud Platform project and credentials.")
-public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {
-  /**
-   * Project id to use when launching jobs.
-   */
-  @Description("Project id. Required when running a Dataflow in the cloud. "
-      + "See https://cloud.google.com/storage/docs/projects for further details.")
-  @Default.InstanceFactory(DefaultProjectFactory.class)
-  String getProject();
-  void setProject(String value);
-
-  /**
-   * This option controls which file to use when attempting to create the credentials using the
-   * service account method.
-   *
-   * <p>This option if specified, needs be combined with the
-   * {@link GcpOptions#getServiceAccountName() serviceAccountName}.
-   */
-  @JsonIgnore
-  @Description("Controls which file to use when attempting to create the credentials "
-      + "using the service account method. This option if specified, needs to be combined with "
-      + "the serviceAccountName option.")
-  String getServiceAccountKeyfile();
-  void setServiceAccountKeyfile(String value);
-
-  /**
-   * This option controls which service account to use when attempting to create the credentials
-   * using the service account method.
-   *
-   * <p>This option if specified, needs be combined with the
-   * {@link GcpOptions#getServiceAccountKeyfile() serviceAccountKeyfile}.
-   */
-  @JsonIgnore
-  @Description("Controls which service account to use when attempting to create the credentials "
-      + "using the service account method. This option if specified, needs to be combined with "
-      + "the serviceAccountKeyfile option.")
-  String getServiceAccountName();
-  void setServiceAccountName(String value);
-
-  /**
-   * This option controls which file to use when attempting to create the credentials
-   * using the OAuth 2 webflow. After the OAuth2 webflow, the credentials will be stored
-   * within credentialDir.
-   */
-  @JsonIgnore
-  @Description("This option controls which file to use when attempting to create the credentials "
-      + "using the OAuth 2 webflow. After the OAuth2 webflow, the credentials will be stored "
-      + "within credentialDir.")
-  String getSecretsFile();
-  void setSecretsFile(String value);
-
-  /**
-   * This option controls which credential store to use when creating the credentials
-   * using the OAuth 2 webflow.
-   */
-  @Description("This option controls which credential store to use when creating the credentials "
-      + "using the OAuth 2 webflow.")
-  @Default.String("cloud_dataflow")
-  String getCredentialId();
-  void setCredentialId(String value);
-
-  /**
-   * Directory for storing dataflow credentials after execution of the OAuth 2 webflow. Defaults
-   * to using the $HOME/.store/data-flow directory.
-   */
-  @Description("Directory for storing dataflow credentials after execution of the OAuth 2 webflow. "
-      + "Defaults to using the $HOME/.store/data-flow directory.")
-  @Default.InstanceFactory(CredentialDirFactory.class)
-  String getCredentialDir();
-  void setCredentialDir(String value);
-
-  /**
-   * Returns the default credential directory of ${user.home}/.store/data-flow.
-   */
-  public static class CredentialDirFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      File home = new File(System.getProperty("user.home"));
-      File store = new File(home, ".store");
-      File dataflow = new File(store, "data-flow");
-      return dataflow.getPath();
-    }
-  }
-
-  /**
-   * The class of the credential factory that should be created and used to create
-   * credentials. If gcpCredential has not been set explicitly, an instance of this class will
-   * be constructed and used as a credential factory.
-   */
-  @Description("The class of the credential factory that should be created and used to create "
-      + "credentials. If gcpCredential has not been set explicitly, an instance of this class will "
-      + "be constructed and used as a credential factory.")
-  @Default.Class(GcpCredentialFactory.class)
-  Class<? extends CredentialFactory> getCredentialFactoryClass();
-  void setCredentialFactoryClass(
-      Class<? extends CredentialFactory> credentialFactoryClass);
-
-  /**
-   * The credential instance that should be used to authenticate against GCP services.
-   * If no credential has been set explicitly, the default is to use the instance factory
-   * that constructs a credential based upon the currently set credentialFactoryClass.
-   */
-  @JsonIgnore
-  @Description("The credential instance that should be used to authenticate against GCP services. "
-      + "If no credential has been set explicitly, the default is to use the instance factory "
-      + "that constructs a credential based upon the currently set credentialFactoryClass.")
-  @Default.InstanceFactory(GcpUserCredentialsFactory.class)
-  @Hidden
-  Credential getGcpCredential();
-  void setGcpCredential(Credential value);
-
-  /**
-   * Attempts to infer the default project based upon the environment this application
-   * is executing within. Currently this only supports getting the default project from gcloud.
-   */
-  public static class DefaultProjectFactory implements DefaultValueFactory<String> {
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultProjectFactory.class);
-
-    @Override
-    public String create(PipelineOptions options) {
-      try {
-        File configFile;
-        if (getEnvironment().containsKey("CLOUDSDK_CONFIG")) {
-          configFile = new File(getEnvironment().get("CLOUDSDK_CONFIG"), "properties");
-        } else if (isWindows() && getEnvironment().containsKey("APPDATA")) {
-          configFile = new File(getEnvironment().get("APPDATA"), "gcloud/properties");
-        } else {
-          // New versions of gcloud use this file
-          configFile = new File(
-              System.getProperty("user.home"),
-              ".config/gcloud/configurations/config_default");
-          if (!configFile.exists()) {
-            // Old versions of gcloud use this file
-            configFile = new File(System.getProperty("user.home"), ".config/gcloud/properties");
-          }
-        }
-        String section = null;
-        Pattern projectPattern = Pattern.compile("^project\\s*=\\s*(.*)$");
-        Pattern sectionPattern = Pattern.compile("^\\[(.*)\\]$");
-        for (String line : Files.readLines(configFile, StandardCharsets.UTF_8)) {
-          line = line.trim();
-          if (line.isEmpty() || line.startsWith(";")) {
-            continue;
-          }
-          Matcher matcher = sectionPattern.matcher(line);
-          if (matcher.matches()) {
-            section = matcher.group(1);
-          } else if (section == null || section.equals("core")) {
-            matcher = projectPattern.matcher(line);
-            if (matcher.matches()) {
-              String project = matcher.group(1).trim();
-              LOG.info("Inferred default GCP project '{}' from gcloud. If this is the incorrect "
-                  + "project, please cancel this Pipeline and specify the command-line "
-                  + "argument --project.", project);
-              return project;
-            }
-          }
-        }
-      } catch (IOException expected) {
-        LOG.debug("Failed to find default project.", expected);
-      }
-      // return null if can't determine
-      return null;
-    }
-
-    /**
-     * Returns true if running on the Windows OS.
-     */
-    private static boolean isWindows() {
-      return System.getProperty("os.name").toLowerCase(Locale.ENGLISH).contains("windows");
-    }
-
-    /**
-     * Used to mock out getting environment variables.
-     */
-    @VisibleForTesting
-    Map<String, String> getEnvironment() {
-        return System.getenv();
-    }
-  }
-
-  /**
-   * Attempts to load the GCP credentials. See
-   * {@link CredentialFactory#getCredential()} for more details.
-   */
-  public static class GcpUserCredentialsFactory implements DefaultValueFactory<Credential> {
-    @Override
-    public Credential create(PipelineOptions options) {
-      GcpOptions gcpOptions = options.as(GcpOptions.class);
-      try {
-        CredentialFactory factory = InstanceBuilder.ofType(CredentialFactory.class)
-            .fromClass(gcpOptions.getCredentialFactoryClass())
-            .fromFactoryMethod("fromOptions")
-            .withArg(PipelineOptions.class, options)
-            .build();
-        return factory.getCredential();
-      } catch (IOException | GeneralSecurityException e) {
-        throw new RuntimeException("Unable to obtain credential", e);
-      }
-    }
-  }
-
-  /**
-   * The token server URL to use for OAuth 2 authentication. Normally, the default is sufficient,
-   * but some specialized use cases may want to override this value.
-   */
-  @Description("The token server URL to use for OAuth 2 authentication. Normally, the default "
-      + "is sufficient, but some specialized use cases may want to override this value.")
-  @Default.String(GoogleOAuthConstants.TOKEN_SERVER_URL)
-  @Hidden
-  String getTokenServerUrl();
-  void setTokenServerUrl(String value);
-
-  /**
-   * The authorization server URL to use for OAuth 2 authentication. Normally, the default is
-   * sufficient, but some specialized use cases may want to override this value.
-   */
-  @Description("The authorization server URL to use for OAuth 2 authentication. Normally, the "
-      + "default is sufficient, but some specialized use cases may want to override this value.")
-  @Default.String(GoogleOAuthConstants.AUTHORIZATION_SERVER_URL)
-  @Hidden
-  String getAuthorizationServerEncodedUrl();
-  void setAuthorizationServerEncodedUrl(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
deleted file mode 100644
index 130310a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
+++ /dev/null
@@ -1,114 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.util.AppEngineEnvironment;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Options used to configure Google Cloud Storage.
- */
-public interface GcsOptions extends
-    ApplicationNameOptions, GcpOptions, PipelineOptions {
-  /**
-   * The GcsUtil instance that should be used to communicate with Google Cloud Storage.
-   */
-  @JsonIgnore
-  @Description("The GcsUtil instance that should be used to communicate with Google Cloud Storage.")
-  @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class)
-  @Hidden
-  GcsUtil getGcsUtil();
-  void setGcsUtil(GcsUtil value);
-
-  /**
-   * The ExecutorService instance to use to create threads, can be overridden to specify an
-   * ExecutorService that is compatible with the users environment. If unset, the
-   * default is to create an ExecutorService with an unbounded number of threads; this
-   * is compatible with Google AppEngine.
-   */
-  @JsonIgnore
-  @Description("The ExecutorService instance to use to create multiple threads. Can be overridden "
-      + "to specify an ExecutorService that is compatible with the users environment. If unset, "
-      + "the default is to create an ExecutorService with an unbounded number of threads; this "
-      + "is compatible with Google AppEngine.")
-  @Default.InstanceFactory(ExecutorServiceFactory.class)
-  @Hidden
-  ExecutorService getExecutorService();
-  void setExecutorService(ExecutorService value);
-
-  /**
-   * GCS endpoint to use. If unspecified, uses the default endpoint.
-   */
-  @JsonIgnore
-  @Hidden
-  @Description("The URL for the GCS API.")
-  String getGcsEndpoint();
-  void setGcsEndpoint(String value);
-
-  /**
-   * The buffer size (in bytes) to use when uploading files to GCS. Please see the documentation for
-   * {@link AbstractGoogleAsyncWriteChannel#setUploadBufferSize} for more information on the
-   * restrictions and performance implications of this value.
-   */
-  @Description("The buffer size (in bytes) to use when uploading files to GCS. Please see the "
-      + "documentation for AbstractGoogleAsyncWriteChannel.setUploadBufferSize for more "
-      + "information on the restrictions and performance implications of this value.\n\n"
-      + "https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/util/src/main/java/"
-      + "com/google/cloud/hadoop/util/AbstractGoogleAsyncWriteChannel.java")
-  Integer getGcsUploadBufferSizeBytes();
-  void setGcsUploadBufferSizeBytes(Integer bytes);
-
-  /**
-   * Returns the default {@link ExecutorService} to use within the Dataflow SDK. The
-   * {@link ExecutorService} is compatible with AppEngine.
-   */
-  public static class ExecutorServiceFactory implements DefaultValueFactory<ExecutorService> {
-    @SuppressWarnings("deprecation")  // IS_APP_ENGINE is deprecated for internal use only.
-    @Override
-    public ExecutorService create(PipelineOptions options) {
-      ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder();
-      threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory());
-      if (!AppEngineEnvironment.IS_APP_ENGINE) {
-        // AppEngine doesn't allow modification of threads to be daemon threads.
-        threadFactoryBuilder.setDaemon(true);
-      }
-      /* The SDK requires an unbounded thread pool because a step may create X writers
-       * each requiring their own thread to perform the writes otherwise a writer may
-       * block causing deadlock for the step because the writers buffer is full.
-       * Also, the MapTaskExecutor launches the steps in reverse order and completes
-       * them in forward order thus requiring enough threads so that each step's writers
-       * can be active.
-       */
-      return new ThreadPoolExecutor(
-          0, Integer.MAX_VALUE, // Allow an unlimited number of re-usable threads.
-          Long.MAX_VALUE, TimeUnit.NANOSECONDS, // Keep non-core threads alive forever.
-          new SynchronousQueue<Runnable>(),
-          threadFactoryBuilder.build());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java
deleted file mode 100644
index 1863141..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java
+++ /dev/null
@@ -1,88 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.api.client.googleapis.services.AbstractGoogleClient;
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.googleapis.services.GoogleClientRequestInitializer;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * These options configure debug settings for Google API clients created within the Dataflow SDK.
- */
-public interface GoogleApiDebugOptions extends PipelineOptions {
-  /**
-   * This option enables tracing of API calls to Google services used within the
-   * Dataflow SDK. Values are expected in JSON format <code>{"ApiName":"TraceDestination",...}
-   * </code> where the {@code ApiName} represents the request classes canonical name. The
-   * {@code TraceDestination} is a logical trace consumer to whom the trace will be reported.
-   * Typically, "producer" is the right destination to use: this makes API traces available to the
-   * team offering the API. Note that by enabling this option, the contents of the requests to and
-   * from Google Cloud services will be made available to Google. For example, by specifying
-   * <code>{"Dataflow":"producer"}</code>, all calls to the Dataflow service will be made available
-   * to Google, specifically to the Google Cloud Dataflow team.
-   */
-  @Description("This option enables tracing of API calls to Google services used within the "
-      + "Dataflow SDK. Values are expected in JSON format {\"ApiName\":\"TraceDestination\",...} "
-      + "where the ApiName represents the request classes canonical name. The TraceDestination is "
-      + "a logical trace consumer to whom the trace will be reported. Typically, \"producer\" is "
-      + "the right destination to use: this makes API traces available to the team offering the "
-      + "API. Note that by enabling this option, the contents of the requests to and from "
-      + "Google Cloud services will be made available to Google. For example, by specifying "
-      + "{\"Dataflow\":\"producer\"}, all calls to the Dataflow service will be made available to "
-      + "Google, specifically to the Google Cloud Dataflow team.")
-  GoogleApiTracer getGoogleApiTrace();
-  void setGoogleApiTrace(GoogleApiTracer commands);
-
-  /**
-   * A {@link GoogleClientRequestInitializer} that adds the trace destination to Google API calls.
-   */
-  public static class GoogleApiTracer extends HashMap<String, String>
-      implements GoogleClientRequestInitializer {
-    /**
-     * Creates a {@link GoogleApiTracer} that sets the trace destination on all
-     * calls that match the given client type.
-     */
-    public GoogleApiTracer addTraceFor(AbstractGoogleClient client, String traceDestination) {
-      put(client.getClass().getCanonicalName(), traceDestination);
-      return this;
-    }
-
-    /**
-     * Creates a {@link GoogleApiTracer} that sets the trace {@code traceDestination} on all
-     * calls that match for the given request type.
-     */
-    public GoogleApiTracer addTraceFor(
-        AbstractGoogleClientRequest<?> request, String traceDestination) {
-      put(request.getClass().getCanonicalName(), traceDestination);
-      return this;
-    }
-
-    @Override
-    public void initialize(AbstractGoogleClientRequest<?> request) throws IOException {
-      for (Map.Entry<String, String> entry : this.entrySet()) {
-        if (request.getClass().getCanonicalName().contains(entry.getKey())) {
-          request.set("$trace", entry.getValue());
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java
deleted file mode 100644
index fd25db8..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.options;
-
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * Methods and/or interfaces annotated with {@code @Hidden} will be suppressed from
- * being output when {@code --help} is specified on the command-line.
- */
-@Target({ElementType.METHOD, ElementType.TYPE})
-@Retention(RetentionPolicy.RUNTIME)
-@Documented
-public @interface Hidden {
-}


[26/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java
deleted file mode 100644
index d2c08c8..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java
+++ /dev/null
@@ -1,483 +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 com.google.cloud.dataflow.sdk.util;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-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 com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpResponse;
-import com.google.api.client.http.HttpStatusCodes;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.LowLevelHttpRequest;
-import com.google.api.client.json.GenericJson;
-import com.google.api.client.json.Json;
-import com.google.api.client.json.JsonFactory;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.client.testing.http.HttpTesting;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
-import com.google.cloud.dataflow.sdk.util.PackageUtil.PackageAttributes;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import com.google.common.io.LineReader;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-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 org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.channels.Channels;
-import java.nio.channels.Pipe;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.regex.Pattern;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-/** Tests for PackageUtil. */
-@RunWith(JUnit4.class)
-public class PackageUtilTest {
-  @Rule public ExpectedLogs logged = ExpectedLogs.none(PackageUtil.class);
-  @Rule
-  public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Rule
-  public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper();
-
-  @Mock
-  GcsUtil mockGcsUtil;
-
-  // 128 bits, base64 encoded is 171 bits, rounds to 22 bytes
-  private static final String HASH_PATTERN = "[a-zA-Z0-9+-]{22}";
-
-  // Hamcrest matcher to assert a string matches a pattern
-  private static class RegexMatcher extends BaseMatcher<String> {
-    private final Pattern pattern;
-
-    public RegexMatcher(String regex) {
-      this.pattern = Pattern.compile(regex);
-    }
-
-    @Override
-    public boolean matches(Object o) {
-      if (!(o instanceof String)) {
-        return false;
-      }
-      return pattern.matcher((String) o).matches();
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText(String.format("matches regular expression %s", pattern));
-    }
-
-    public static RegexMatcher matches(String regex) {
-      return new RegexMatcher(regex);
-    }
-  }
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-
-    GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
-    pipelineOptions.setGcsUtil(mockGcsUtil);
-
-    IOChannelUtils.registerStandardIOFactories(pipelineOptions);
-  }
-
-  private File makeFileWithContents(String name, String contents) throws Exception {
-    File tmpFile = tmpFolder.newFile(name);
-    Files.write(contents, tmpFile, StandardCharsets.UTF_8);
-    tmpFile.setLastModified(0);  // required for determinism with directories
-    return tmpFile;
-  }
-
-  static final String STAGING_PATH = GcsPath.fromComponents("somebucket", "base/path").toString();
-  private static PackageAttributes makePackageAttributes(File file, String overridePackageName) {
-    return PackageUtil.createPackageAttributes(file, STAGING_PATH, overridePackageName);
-  }
-
-  @Test
-  public void testFileWithExtensionPackageNamingAndSize() throws Exception {
-    String contents = "This is a test!";
-    File tmpFile = makeFileWithContents("file.txt", contents);
-    PackageAttributes attr = makePackageAttributes(tmpFile, null);
-    DataflowPackage target = attr.getDataflowPackage();
-
-    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
-    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
-    assertThat(attr.getSize(), equalTo((long) contents.length()));
-  }
-
-  @Test
-  public void testPackageNamingWithFileNoExtension() throws Exception {
-    File tmpFile = makeFileWithContents("file", "This is a test!");
-    DataflowPackage target = makePackageAttributes(tmpFile, null).getDataflowPackage();
-
-    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN));
-    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
-  }
-
-  @Test
-  public void testPackageNamingWithDirectory() throws Exception {
-    File tmpDirectory = tmpFolder.newFolder("folder");
-    DataflowPackage target = makePackageAttributes(tmpDirectory, null).getDataflowPackage();
-
-    assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
-    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
-  }
-
-  @Test
-  public void testPackageNamingWithFilesHavingSameContentsAndSameNames() throws Exception {
-    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
-    makeFileWithContents("folder1/folderA/sameName", "This is a test!");
-    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
-    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
-    makeFileWithContents("folder2/folderA/sameName", "This is a test!");
-    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
-    assertEquals(target1.getName(), target2.getName());
-    assertEquals(target1.getLocation(), target2.getLocation());
-  }
-
-  @Test
-  public void testPackageNamingWithFilesHavingSameContentsButDifferentNames() throws Exception {
-    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
-    makeFileWithContents("folder1/folderA/uniqueName1", "This is a test!");
-    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
-    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
-    makeFileWithContents("folder2/folderA/uniqueName2", "This is a test!");
-    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
-    assertNotEquals(target1.getName(), target2.getName());
-    assertNotEquals(target1.getLocation(), target2.getLocation());
-  }
-
-  @Test
-  public void testPackageNamingWithDirectoriesHavingSameContentsButDifferentNames()
-      throws Exception {
-    File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
-    tmpFolder.newFolder("folder1", "folderA", "uniqueName1");
-    DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
-    File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
-    tmpFolder.newFolder("folder2", "folderA", "uniqueName2");
-    DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
-    assertNotEquals(target1.getName(), target2.getName());
-    assertNotEquals(target1.getLocation(), target2.getLocation());
-  }
-
-  @Test
-  public void testPackageUploadWithLargeClasspathLogsWarning() throws Exception {
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    // all files will be present and cached so no upload needed.
-    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
-
-    List<String> classpathElements = Lists.newLinkedList();
-    for (int i = 0; i < 1005; ++i) {
-      String eltName = "element" + i;
-      classpathElements.add(eltName + '=' + tmpFile.getAbsolutePath());
-    }
-
-    PackageUtil.stageClasspathElements(classpathElements, STAGING_PATH);
-
-    logged.verifyWarn("Your classpath contains 1005 elements, which Google Cloud Dataflow");
-  }
-
-  @Test
-  public void testPackageUploadWithFileSucceeds() throws Exception {
-    Pipe pipe = Pipe.open();
-    String contents = "This is a test!";
-    File tmpFile = makeFileWithContents("file.txt", contents);
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
-    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
-        ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
-    DataflowPackage target = Iterables.getOnlyElement(targets);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-    verifyNoMoreInteractions(mockGcsUtil);
-
-    assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
-    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
-    assertThat(new LineReader(Channels.newReader(pipe.source(), "UTF-8")).readLine(),
-        equalTo(contents));
-  }
-
-  @Test
-  public void testPackageUploadWithDirectorySucceeds() throws Exception {
-    Pipe pipe = Pipe.open();
-    File tmpDirectory = tmpFolder.newFolder("folder");
-    tmpFolder.newFolder("folder", "empty_directory");
-    tmpFolder.newFolder("folder", "directory");
-    makeFileWithContents("folder/file.txt", "This is a test!");
-    makeFileWithContents("folder/directory/file.txt", "This is also a test!");
-
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
-    PackageUtil.stageClasspathElements(
-        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-    verifyNoMoreInteractions(mockGcsUtil);
-
-    ZipInputStream inputStream = new ZipInputStream(Channels.newInputStream(pipe.source()));
-    List<String> zipEntryNames = new ArrayList<>();
-    for (ZipEntry entry = inputStream.getNextEntry(); entry != null;
-        entry = inputStream.getNextEntry()) {
-      zipEntryNames.add(entry.getName());
-    }
-
-    assertThat(zipEntryNames,
-        containsInAnyOrder("directory/file.txt", "empty_directory/", "file.txt"));
-  }
-
-  @Test
-  public void testPackageUploadWithEmptyDirectorySucceeds() throws Exception {
-    Pipe pipe = Pipe.open();
-    File tmpDirectory = tmpFolder.newFolder("folder");
-
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
-    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
-        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
-    DataflowPackage target = Iterables.getOnlyElement(targets);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-    verifyNoMoreInteractions(mockGcsUtil);
-
-    assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
-    assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
-    assertNull(new ZipInputStream(Channels.newInputStream(pipe.source())).getNextEntry());
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception {
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
-        .thenThrow(new IOException("Fake Exception: Upload error"));
-
-    try {
-      PackageUtil.stageClasspathElements(
-          ImmutableList.of(tmpFile.getAbsolutePath()),
-          STAGING_PATH, fastNanoClockAndSleeper);
-    } finally {
-      verify(mockGcsUtil).fileSize(any(GcsPath.class));
-      verify(mockGcsUtil, times(5)).create(any(GcsPath.class), anyString());
-      verifyNoMoreInteractions(mockGcsUtil);
-    }
-  }
-
-  @Test
-  public void testPackageUploadFailsWithPermissionsErrorGivesDetailedMessage() throws Exception {
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
-        .thenThrow(new IOException("Failed to write to GCS path " + STAGING_PATH,
-            googleJsonResponseException(
-                HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Permission denied", "Test message")));
-
-    try {
-      PackageUtil.stageClasspathElements(
-          ImmutableList.of(tmpFile.getAbsolutePath()),
-          STAGING_PATH, fastNanoClockAndSleeper);
-      fail("Expected RuntimeException");
-    } catch (RuntimeException e) {
-      assertTrue("Expected IOException containing detailed message.",
-          e.getCause() instanceof IOException);
-      assertThat(e.getCause().getMessage(),
-          Matchers.allOf(
-              Matchers.containsString("Uploaded failed due to permissions error"),
-              Matchers.containsString(
-                  "Stale credentials can be resolved by executing 'gcloud auth login'")));
-    } finally {
-      verify(mockGcsUtil).fileSize(any(GcsPath.class));
-      verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-      verifyNoMoreInteractions(mockGcsUtil);
-    }
-  }
-
-  @Test
-  public void testPackageUploadEventuallySucceeds() throws Exception {
-    Pipe pipe = Pipe.open();
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
-        .thenThrow(new IOException("Fake Exception: 410 Gone")) // First attempt fails
-        .thenReturn(pipe.sink());                               // second attempt succeeds
-
-    try {
-      PackageUtil.stageClasspathElements(
-                                              ImmutableList.of(tmpFile.getAbsolutePath()),
-                                              STAGING_PATH,
-                                              fastNanoClockAndSleeper);
-    } finally {
-      verify(mockGcsUtil).fileSize(any(GcsPath.class));
-      verify(mockGcsUtil, times(2)).create(any(GcsPath.class), anyString());
-      verifyNoMoreInteractions(mockGcsUtil);
-    }
-  }
-
-  @Test
-  public void testPackageUploadIsSkippedWhenFileAlreadyExists() throws Exception {
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
-
-    PackageUtil.stageClasspathElements(
-        ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verifyNoMoreInteractions(mockGcsUtil);
-  }
-
-  @Test
-  public void testPackageUploadIsNotSkippedWhenSizesAreDifferent() throws Exception {
-    Pipe pipe = Pipe.open();
-    File tmpDirectory = tmpFolder.newFolder("folder");
-    tmpFolder.newFolder("folder", "empty_directory");
-    tmpFolder.newFolder("folder", "directory");
-    makeFileWithContents("folder/file.txt", "This is a test!");
-    makeFileWithContents("folder/directory/file.txt", "This is also a test!");
-    when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(Long.MAX_VALUE);
-    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
-    PackageUtil.stageClasspathElements(
-        ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-    verifyNoMoreInteractions(mockGcsUtil);
-  }
-
-  @Test
-  public void testPackageUploadWithExplicitPackageName() throws Exception {
-    Pipe pipe = Pipe.open();
-    File tmpFile = makeFileWithContents("file.txt", "This is a test!");
-    final String overriddenName = "alias.txt";
-
-    when(mockGcsUtil.fileSize(any(GcsPath.class)))
-        .thenThrow(new FileNotFoundException("some/path"));
-    when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
-    List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
-        ImmutableList.of(overriddenName + "=" + tmpFile.getAbsolutePath()), STAGING_PATH);
-    DataflowPackage target = Iterables.getOnlyElement(targets);
-
-    verify(mockGcsUtil).fileSize(any(GcsPath.class));
-    verify(mockGcsUtil).create(any(GcsPath.class), anyString());
-    verifyNoMoreInteractions(mockGcsUtil);
-
-    assertThat(target.getName(), equalTo(overriddenName));
-    assertThat(target.getLocation(),
-        RegexMatcher.matches(STAGING_PATH + "/file-" + HASH_PATTERN + ".txt"));
-  }
-
-  @Test
-  public void testPackageUploadIsSkippedWithNonExistentResource() throws Exception {
-    String nonExistentFile =
-        IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "non-existent-file");
-    assertEquals(Collections.EMPTY_LIST, PackageUtil.stageClasspathElements(
-        ImmutableList.of(nonExistentFile), STAGING_PATH));
-  }
-
-  /**
-   * Builds a fake GoogleJsonResponseException for testing API error handling.
-   */
-  private static GoogleJsonResponseException googleJsonResponseException(
-      final int status, final String reason, final String message) throws IOException {
-    final JsonFactory jsonFactory = new JacksonFactory();
-    HttpTransport transport = new MockHttpTransport() {
-      @Override
-      public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
-        ErrorInfo errorInfo = new ErrorInfo();
-        errorInfo.setReason(reason);
-        errorInfo.setMessage(message);
-        errorInfo.setFactory(jsonFactory);
-        GenericJson error = new GenericJson();
-        error.set("code", status);
-        error.set("errors", Arrays.asList(errorInfo));
-        error.setFactory(jsonFactory);
-        GenericJson errorResponse = new GenericJson();
-        errorResponse.set("error", error);
-        errorResponse.setFactory(jsonFactory);
-        return new MockLowLevelHttpRequest().setResponse(
-            new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
-            .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
-        }
-    };
-    HttpRequest request =
-        transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
-    request.setThrowExceptionOnExecuteError(false);
-    HttpResponse response = request.execute();
-    return GoogleJsonResponseException.from(jsonFactory, response);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
new file mode 100644
index 0000000..7788b5b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.io;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.List;
+
+/**
+ * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write transforms.
+ */
+@RunWith(JUnit4.class)
+public class DataflowTextIOTest {
+
+  private TestDataflowPipelineOptions buildTestPipelineOptions() {
+    TestDataflowPipelineOptions options =
+        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+    options.setGcpCredential(new TestCredential());
+    return options;
+  }
+
+  private GcsUtil buildMockGcsUtil() throws IOException {
+    GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
+
+    // Any request to open gets a new bogus channel
+    Mockito
+        .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
+        .then(new Answer<SeekableByteChannel>() {
+          @Override
+          public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+            return FileChannel.open(
+                Files.createTempFile("channel-", ".tmp"),
+                StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+          }
+        });
+
+    // Any request for expansion returns a list containing the original GcsPath
+    // This is required to pass validation that occurs in TextIO during apply()
+    Mockito
+        .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
+        .then(new Answer<List<GcsPath>>() {
+          @Override
+          public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+            return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+          }
+        });
+
+    return mockGcsUtil;
+  }
+
+  /**
+   * This tests a few corner cases that should not crash.
+   */
+  @Test
+  public void testGoodWildcards() throws Exception {
+    TestDataflowPipelineOptions options = buildTestPipelineOptions();
+    options.setGcsUtil(buildMockGcsUtil());
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    applyRead(pipeline, "gs://bucket/foo");
+    applyRead(pipeline, "gs://bucket/foo/");
+    applyRead(pipeline, "gs://bucket/foo/*");
+    applyRead(pipeline, "gs://bucket/foo/?");
+    applyRead(pipeline, "gs://bucket/foo/[0-9]");
+    applyRead(pipeline, "gs://bucket/foo/*baz*");
+    applyRead(pipeline, "gs://bucket/foo/*baz?");
+    applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
+    applyRead(pipeline, "gs://bucket/foo/baz/*");
+    applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
+    applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
+    applyRead(pipeline, "gs://bucket/foo*/baz");
+    applyRead(pipeline, "gs://bucket/foo?/baz");
+    applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
+
+    // Check that running doesn't fail.
+    pipeline.run();
+  }
+
+  private void applyRead(Pipeline pipeline, String path) {
+    pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
new file mode 100644
index 0000000..1b5a3c7
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.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 com.google.cloud.dataflow.sdk.options;
+
+import static org.hamcrest.Matchers.hasEntry;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowPipelineDebugOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineDebugOptionsTest {
+  @Test
+  public void testTransformNameMapping() throws Exception {
+    DataflowPipelineDebugOptions options = PipelineOptionsFactory
+        .fromArgs(new String[]{"--transformNameMapping={\"a\":\"b\",\"foo\":\"\",\"bar\":\"baz\"}"})
+        .as(DataflowPipelineDebugOptions.class);
+    assertEquals(3, options.getTransformNameMapping().size());
+    assertThat(options.getTransformNameMapping(), hasEntry("a", "b"));
+    assertThat(options.getTransformNameMapping(), hasEntry("foo", ""));
+    assertThat(options.getTransformNameMapping(), hasEntry("bar", "baz"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
new file mode 100644
index 0000000..eff79bb
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.cloud.dataflow.sdk.testing.ResetDateTimeProvider;
+import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowPipelineOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineOptionsTest {
+  @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
+  @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider();
+
+  @Test
+  public void testJobNameIsSet() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    assertEquals("TestJobName", options.getJobName());
+  }
+
+  @Test
+  public void testUserNameIsNotSet() {
+    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+    System.getProperties().remove("user.name");
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setAppName("TestApplication");
+    assertEquals("testapplication--1208190706", options.getJobName());
+    assertTrue(options.getJobName().length() <= 40);
+  }
+
+  @Test
+  public void testAppNameAndUserNameAreLong() {
+    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+    System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setAppName("1234567890123456789012345678901234567890");
+    assertEquals(
+        "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706",
+        options.getJobName());
+  }
+
+  @Test
+  public void testAppNameIsLong() {
+    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+    System.getProperties().put("user.name", "abcde");
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setAppName("1234567890123456789012345678901234567890");
+    assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName());
+  }
+
+  @Test
+  public void testUserNameIsLong() {
+    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+    System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setAppName("1234567890");
+    assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName());
+  }
+
+  @Test
+  public void testUtf8UserNameAndApplicationNameIsNormalized() {
+    resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+    System.getProperties().put("user.name", "ði ıntəˈnæʃənəl ");
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn");
+    assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
new file mode 100644
index 0000000..1420273
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.hamcrest.Matchers;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link DataflowProfilingOptions}.
+ */
+@RunWith(JUnit4.class)
+public class DataflowProfilingOptionsTest {
+
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+
+  @Test
+  public void testOptionsObject() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {
+        "--enableProfilingAgent", "--profilingAgentConfiguration={\"interval\": 21}"})
+        .as(DataflowPipelineOptions.class);
+    assertTrue(options.getEnableProfilingAgent());
+
+    String json = MAPPER.writeValueAsString(options);
+    assertThat(json, Matchers.containsString(
+        "\"profilingAgentConfiguration\":{\"interval\":21}"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
new file mode 100644
index 0000000..b752f3d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.WARN;
+import static org.junit.Assert.assertEquals;
+
+import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
+import com.google.common.collect.ImmutableMap;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+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 DataflowWorkerLoggingOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowWorkerLoggingOptionsTest {
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+
+  @Test
+  public void testWorkerLogLevelOverrideWithInvalidLogLevel() {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Unsupported log level");
+    WorkerLogLevelOverrides.from(ImmutableMap.of("Name", "FakeLevel"));
+  }
+
+  @Test
+  public void testWorkerLogLevelOverrideForClass() throws Exception {
+    assertEquals("{\"org.junit.Test\":\"WARN\"}",
+        MAPPER.writeValueAsString(
+            new WorkerLogLevelOverrides().addOverrideForClass(Test.class, WARN)));
+  }
+
+  @Test
+  public void testWorkerLogLevelOverrideForPackage() throws Exception {
+    assertEquals("{\"org.junit\":\"WARN\"}",
+        MAPPER.writeValueAsString(
+            new WorkerLogLevelOverrides().addOverrideForPackage(Test.class.getPackage(), WARN)));
+  }
+
+  @Test
+  public void testWorkerLogLevelOverrideForName() throws Exception {
+    assertEquals("{\"A\":\"WARN\"}",
+        MAPPER.writeValueAsString(
+            new WorkerLogLevelOverrides().addOverrideForName("A", WARN)));
+  }
+
+  @Test
+  public void testSerializationAndDeserializationOf() throws Exception {
+    String testValue = "{\"A\":\"WARN\"}";
+    assertEquals(testValue,
+        MAPPER.writeValueAsString(
+            MAPPER.readValue(testValue, WorkerLogLevelOverrides.class)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
new file mode 100644
index 0000000..0322426
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
@@ -0,0 +1,302 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
+import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+
+import org.hamcrest.Description;
+import org.hamcrest.Factory;
+import org.hamcrest.Matcher;
+import org.hamcrest.TypeSafeMatcher;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for BlockingDataflowPipelineRunner.
+ */
+@RunWith(JUnit4.class)
+public class BlockingDataflowPipelineRunnerTest {
+
+  @Rule
+  public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class);
+
+  @Rule
+  public ExpectedException expectedThrown = ExpectedException.none();
+
+  /**
+   * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
+   * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
+   */
+  private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
+      extends TypeSafeMatcher<T> {
+
+    private final Matcher<DataflowPipelineJob> matcher;
+
+    public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T ex) {
+      return matcher.matches(ex.getJob());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("job ");
+        matcher.describeMismatch(item.getMessage(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("exception with job matching ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowJobException> Matcher<T> expectJob(
+        Matcher<DataflowPipelineJob> matcher) {
+      return new DataflowJobExceptionMatcher<T>(matcher);
+    }
+  }
+
+  /**
+   * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
+   * to the return value of {@link DataflowPipelineJob#getJobId()}.
+   */
+  private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
+
+    private final Matcher<String> matcher;
+
+    public JobIdMatcher(Matcher<String> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T job) {
+      return matcher.matches(job.getJobId());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("jobId ");
+        matcher.describeMismatch(item.getJobId(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("job with jobId ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
+      return new JobIdMatcher<T>(equalTo(jobId));
+    }
+  }
+
+  /**
+   * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
+   * {@link Matcher} to the {@link DataflowPipelineJob} returned by
+   * {@link DataflowJobUpdatedException#getReplacedByJob()}.
+   */
+  private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
+      extends TypeSafeMatcher<T> {
+
+    private final Matcher<DataflowPipelineJob> matcher;
+
+    public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T ex) {
+      return matcher.matches(ex.getReplacedByJob());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("job ");
+        matcher.describeMismatch(item.getMessage(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("exception with replacedByJob() ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
+        Matcher<DataflowPipelineJob> matcher) {
+      return new ReplacedByJobMatcher<T>(matcher);
+    }
+  }
+
+  /**
+   * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
+   * that will immediately terminate in the provided {@code terminalState}.
+   *
+   * <p>The return value may be further mocked.
+   */
+  private DataflowPipelineJob createMockJob(
+      String projectId, String jobId, State terminalState) throws Exception {
+    DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
+    when(mockJob.getProjectId()).thenReturn(projectId);
+    when(mockJob.getJobId()).thenReturn(jobId);
+    when(mockJob.waitToFinish(
+        anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
+        .thenReturn(terminalState);
+    return mockJob;
+  }
+
+  /**
+   * Returns a {@link BlockingDataflowPipelineRunner} that will return the provided a job to return.
+   * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
+   */
+  private BlockingDataflowPipelineRunner createMockRunner(DataflowPipelineJob job)
+      throws Exception {
+    DataflowPipelineRunner mockRunner = mock(DataflowPipelineRunner.class);
+    TestDataflowPipelineOptions options =
+        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+    options.setProject(job.getProjectId());
+
+    when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
+
+    return new BlockingDataflowPipelineRunner(mockRunner, options);
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} returns normally when a job terminates in
+   * the {@link State#DONE DONE} state.
+   */
+  @Test
+  public void testJobDoneComplete() throws Exception {
+    createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
+        .run(DirectPipeline.createForTest());
+    expectedLogs.verifyInfo("Job finished with status DONE");
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#FAILED FAILED} state.
+   */
+  @Test
+  public void testFailedJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobExecutionException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testFailedJob-jobId")));
+    createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
+        .run(DirectPipeline.createForTest());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
+   */
+  @Test
+  public void testCancelledJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobCancelledException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testCancelledJob-jobId")));
+    createMockRunner(
+        createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
+        .run(DirectPipeline.createForTest());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#UPDATED UPDATED} state.
+   */
+  @Test
+  public void testUpdatedJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobUpdatedException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
+    expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
+        JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
+    DataflowPipelineJob job =
+        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
+    DataflowPipelineJob replacedByJob =
+        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
+    when(job.getReplacedByJob()).thenReturn(replacedByJob);
+    createMockRunner(job).run(DirectPipeline.createForTest());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
+   * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
+   * is an old SDK relative the service).
+   */
+  @Test
+  public void testUnknownJobThrowsException() throws Exception {
+    expectedThrown.expect(IllegalStateException.class);
+    createMockRunner(
+        createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
+        .run(DirectPipeline.createForTest());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+   * when a job returns a {@code null} state, indicating that it failed to contact the service,
+   * including all of its built-in resilience logic.
+   */
+  @Test
+  public void testNullJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowServiceException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testNullJob-jobId")));
+    createMockRunner(
+        createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
+        .run(DirectPipeline.createForTest());
+  }
+
+  @Test
+  public void testToString() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    assertEquals("BlockingDataflowPipelineRunner#TestJobName",
+        BlockingDataflowPipelineRunner.fromOptions(options).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
new file mode 100644
index 0000000..764c0cb
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
@@ -0,0 +1,605 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
+import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSetMultimap;
+
+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 java.io.IOException;
+import java.math.BigDecimal;
+import java.net.SocketTimeoutException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for DataflowPipelineJob.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineJobTest {
+  private static final String PROJECT_ID = "someProject";
+  private static final String JOB_ID = "1234";
+
+  @Mock
+  private Dataflow mockWorkflowClient;
+  @Mock
+  private Dataflow.Projects mockProjects;
+  @Mock
+  private Dataflow.Projects.Jobs mockJobs;
+  @Rule
+  public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+
+    when(mockWorkflowClient.projects()).thenReturn(mockProjects);
+    when(mockProjects.jobs()).thenReturn(mockJobs);
+  }
+
+  /**
+   * Validates that a given time is valid for the total time slept by a
+   * AttemptBoundedExponentialBackOff given the number of retries and
+   * an initial polling interval.
+   *
+   * @param pollingIntervalMillis The initial polling interval given.
+   * @param attempts The number of attempts made
+   * @param timeSleptMillis The amount of time slept by the clock. This is checked
+   * against the valid interval.
+   */
+  void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) {
+    long highSum = 0;
+    long lowSum = 0;
+    for (int i = 1; i < attempts; i++) {
+      double currentInterval =
+          pollingIntervalMillis
+          * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1);
+      double offset =
+          AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval;
+      highSum += Math.round(currentInterval + offset);
+      lowSum += Math.round(currentInterval - offset);
+    }
+    assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum)));
+  }
+
+  @Test
+  public void testWaitToFinishMessagesFail() throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    Job statusResponse = new Job();
+    statusResponse.setCurrentState("JOB_STATE_" + State.DONE.name());
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenReturn(statusResponse);
+
+    MonitoringUtil.JobMessagesHandler jobHandler = mock(MonitoringUtil.JobMessagesHandler.class);
+    Dataflow.Projects.Jobs.Messages mockMessages =
+        mock(Dataflow.Projects.Jobs.Messages.class);
+    Messages.List listRequest = mock(Dataflow.Projects.Jobs.Messages.List.class);
+    when(mockJobs.messages()).thenReturn(mockMessages);
+    when(mockMessages.list(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(listRequest);
+    when(listRequest.execute()).thenThrow(SocketTimeoutException.class);
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+    State state = job.waitToFinish(5, TimeUnit.MINUTES, jobHandler, fastClock, fastClock);
+    assertEquals(null, state);
+  }
+
+  public State mockWaitToFinishInState(State state) throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    Job statusResponse = new Job();
+    statusResponse.setCurrentState("JOB_STATE_" + state.name());
+
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenReturn(statusResponse);
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+    return job.waitToFinish(1, TimeUnit.MINUTES, null, fastClock, fastClock);
+  }
+
+  /**
+   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#DONE DONE}
+   * state is terminal.
+   */
+  @Test
+  public void testWaitToFinishDone() throws Exception {
+    assertEquals(State.DONE, mockWaitToFinishInState(State.DONE));
+  }
+
+  /**
+   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+   * state is terminal.
+   */
+  @Test
+  public void testWaitToFinishFailed() throws Exception {
+    assertEquals(State.FAILED, mockWaitToFinishInState(State.FAILED));
+  }
+
+  /**
+   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+   * state is terminal.
+   */
+  @Test
+  public void testWaitToFinishCancelled() throws Exception {
+    assertEquals(State.CANCELLED, mockWaitToFinishInState(State.CANCELLED));
+  }
+
+  /**
+   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+   * state is terminal.
+   */
+  @Test
+  public void testWaitToFinishUpdated() throws Exception {
+    assertEquals(State.UPDATED, mockWaitToFinishInState(State.UPDATED));
+  }
+
+  @Test
+  public void testWaitToFinishFail() throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenThrow(IOException.class);
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+    long startTime = fastClock.nanoTime();
+    State state = job.waitToFinish(5, TimeUnit.MINUTES, null, fastClock, fastClock);
+    assertEquals(null, state);
+    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+    checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL,
+        DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff);
+  }
+
+  @Test
+  public void testWaitToFinishTimeFail() throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenThrow(IOException.class);
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+    long startTime = fastClock.nanoTime();
+    State state = job.waitToFinish(4, TimeUnit.MILLISECONDS, null, fastClock, fastClock);
+    assertEquals(null, state);
+    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+    // Should only sleep for the 4 ms remaining.
+    assertEquals(timeDiff, 4L);
+  }
+
+  @Test
+  public void testGetStateReturnsServiceState() throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    Job statusResponse = new Job();
+    statusResponse.setCurrentState("JOB_STATE_" + State.RUNNING.name());
+
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenReturn(statusResponse);
+
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+    assertEquals(
+        State.RUNNING,
+        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+  }
+
+  @Test
+  public void testGetStateWithExceptionReturnsUnknown() throws Exception {
+    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+    when(statusRequest.execute()).thenThrow(IOException.class);
+    DataflowAggregatorTransforms dataflowAggregatorTransforms =
+        mock(DataflowAggregatorTransforms.class);
+
+    DataflowPipelineJob job = new DataflowPipelineJob(
+        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+    long startTime = fastClock.nanoTime();
+    assertEquals(
+        State.UNKNOWN,
+        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+    checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL,
+        DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff);
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithNoMetricUpdatesReturnsEmptyValue()
+      throws IOException, AggregatorRetrievalException {
+    Aggregator<?, ?> aggregator = mock(Aggregator.class);
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    JobMetrics jobMetrics = new JobMetrics();
+    when(getMetrics.execute()).thenReturn(jobMetrics);
+
+    jobMetrics.setMetrics(ImmutableList.<MetricUpdate>of());
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    AggregatorValues<?> values = job.getAggregatorValues(aggregator);
+
+    assertThat(values.getValues(), empty());
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithNullMetricUpdatesReturnsEmptyValue()
+      throws IOException, AggregatorRetrievalException {
+    Aggregator<?, ?> aggregator = mock(Aggregator.class);
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    JobMetrics jobMetrics = new JobMetrics();
+    when(getMetrics.execute()).thenReturn(jobMetrics);
+
+    jobMetrics.setMetrics(null);
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    AggregatorValues<?> values = job.getAggregatorValues(aggregator);
+
+    assertThat(values.getValues(), empty());
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithSingleMetricUpdateReturnsSingletonCollection()
+      throws IOException, AggregatorRetrievalException {
+    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+    String aggregatorName = "agg";
+    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    JobMetrics jobMetrics = new JobMetrics();
+    when(getMetrics.execute()).thenReturn(jobMetrics);
+
+    MetricUpdate update = new MetricUpdate();
+    long stepValue = 1234L;
+    update.setScalar(new BigDecimal(stepValue));
+
+    MetricStructuredName structuredName = new MetricStructuredName();
+    structuredName.setName(aggregatorName);
+    structuredName.setContext(ImmutableMap.of("step", stepName));
+    update.setName(structuredName);
+
+    jobMetrics.setMetrics(ImmutableList.of(update));
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+    assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
+    assertThat(values.getValuesAtSteps().size(), equalTo(1));
+    assertThat(values.getValues(), contains(stepValue));
+    assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue)));
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithMultipleMetricUpdatesReturnsCollection()
+      throws IOException, AggregatorRetrievalException {
+    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+    String aggregatorName = "agg";
+    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> otherTransform = mock(PTransform.class);
+    String otherStepName = "s88";
+    String otherFullName = "Spam/Ham/Eggs";
+    AppliedPTransform<?, ?, ?> otherAppliedTransform =
+        appliedPTransform(otherFullName, otherTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(
+                                aggregator, pTransform, aggregator, otherTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(
+            appliedTransform, stepName, otherAppliedTransform, otherStepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    JobMetrics jobMetrics = new JobMetrics();
+    when(getMetrics.execute()).thenReturn(jobMetrics);
+
+    MetricUpdate updateOne = new MetricUpdate();
+    long stepValue = 1234L;
+    updateOne.setScalar(new BigDecimal(stepValue));
+
+    MetricStructuredName structuredNameOne = new MetricStructuredName();
+    structuredNameOne.setName(aggregatorName);
+    structuredNameOne.setContext(ImmutableMap.of("step", stepName));
+    updateOne.setName(structuredNameOne);
+
+    MetricUpdate updateTwo = new MetricUpdate();
+    long stepValueTwo = 1024L;
+    updateTwo.setScalar(new BigDecimal(stepValueTwo));
+
+    MetricStructuredName structuredNameTwo = new MetricStructuredName();
+    structuredNameTwo.setName(aggregatorName);
+    structuredNameTwo.setContext(ImmutableMap.of("step", otherStepName));
+    updateTwo.setName(structuredNameTwo);
+
+    jobMetrics.setMetrics(ImmutableList.of(updateOne, updateTwo));
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+    assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
+    assertThat(values.getValuesAtSteps(), hasEntry(otherFullName, stepValueTwo));
+    assertThat(values.getValuesAtSteps().size(), equalTo(2));
+    assertThat(values.getValues(), containsInAnyOrder(stepValue, stepValueTwo));
+    assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue + stepValueTwo)));
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithUnrelatedMetricUpdateIgnoresUpdate()
+      throws IOException, AggregatorRetrievalException {
+    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+    String aggregatorName = "agg";
+    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    JobMetrics jobMetrics = new JobMetrics();
+    when(getMetrics.execute()).thenReturn(jobMetrics);
+
+    MetricUpdate ignoredUpdate = new MetricUpdate();
+    ignoredUpdate.setScalar(null);
+
+    MetricStructuredName ignoredName = new MetricStructuredName();
+    ignoredName.setName("ignoredAggregator.elementCount.out0");
+    ignoredName.setContext(null);
+    ignoredUpdate.setName(ignoredName);
+
+    jobMetrics.setMetrics(ImmutableList.of(ignoredUpdate));
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+    assertThat(values.getValuesAtSteps().entrySet(), empty());
+    assertThat(values.getValues(), empty());
+  }
+
+  @Test
+  public void testGetAggregatorValuesWithUnusedAggregatorThrowsException()
+      throws AggregatorRetrievalException {
+    Aggregator<?, ?> aggregator = mock(Aggregator.class);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of().asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("not used in this pipeline");
+
+    job.getAggregatorValues(aggregator);
+  }
+
+  @Test
+  public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetrievalException()
+      throws IOException, AggregatorRetrievalException {
+    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+    String aggregatorName = "agg";
+    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+    @SuppressWarnings("unchecked")
+    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+    String stepName = "s1";
+    String fullName = "Foo/Bar/Baz";
+    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+    GetMetrics getMetrics = mock(GetMetrics.class);
+    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+    IOException cause = new IOException();
+    when(getMetrics.execute()).thenThrow(cause);
+
+    Get getState = mock(Get.class);
+    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+    Job modelJob = new Job();
+    when(getState.execute()).thenReturn(modelJob);
+    modelJob.setCurrentState(State.RUNNING.toString());
+
+    DataflowPipelineJob job =
+        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+    thrown.expect(AggregatorRetrievalException.class);
+    thrown.expectCause(is(cause));
+    thrown.expectMessage(aggregator.toString());
+    thrown.expectMessage("when retrieving Aggregator values for");
+
+    job.getAggregatorValues(aggregator);
+  }
+
+  private static class TestAggregator<InT, OutT> implements Aggregator<InT, OutT> {
+    private final CombineFn<InT, ?, OutT> combineFn;
+    private final String name;
+
+    public TestAggregator(CombineFn<InT, ?, OutT> combineFn, String name) {
+      this.combineFn = combineFn;
+      this.name = name;
+    }
+
+    @Override
+    public void addValue(InT value) {
+      throw new AssertionError();
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public CombineFn<InT, ?, OutT> getCombineFn() {
+      return combineFn;
+    }
+  }
+
+  private AppliedPTransform<?, ?, ?> appliedPTransform(
+      String fullName, PTransform<PInput, POutput> transform) {
+    return AppliedPTransform.of(fullName, mock(PInput.class), mock(POutput.class), transform);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
new file mode 100644
index 0000000..4850939
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ServiceLoader;
+
+/** Tests for {@link DataflowPipelineRegistrar}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineRegistrarTest {
+  @Test
+  public void testCorrectOptionsAreReturned() {
+    assertEquals(ImmutableList.of(DataflowPipelineOptions.class,
+                                  BlockingDataflowPipelineOptions.class),
+        new DataflowPipelineRegistrar.Options().getPipelineOptions());
+  }
+
+  @Test
+  public void testCorrectRunnersAreReturned() {
+    assertEquals(ImmutableList.of(DataflowPipelineRunner.class,
+                                  BlockingDataflowPipelineRunner.class),
+        new DataflowPipelineRegistrar.Runner().getPipelineRunners());
+  }
+
+  @Test
+  public void testServiceLoaderForOptions() {
+    for (PipelineOptionsRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
+      if (registrar instanceof DataflowPipelineRegistrar.Options) {
+        return;
+      }
+    }
+    fail("Expected to find " + DataflowPipelineRegistrar.Options.class);
+  }
+
+  @Test
+  public void testServiceLoaderForRunner() {
+    for (PipelineRunnerRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
+      if (registrar instanceof DataflowPipelineRegistrar.Runner) {
+        return;
+      }
+    }
+    fail("Expected to find " + DataflowPipelineRegistrar.Runner.class);
+  }
+}


[44/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..c61873c
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
@@ -0,0 +1,164 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+
+import java.io.IOException;
+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
+ * 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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
+ * and then exits.
+ */
+public class StreamingWordExtract {
+
+  /** A DoFn that tokenizes lines of text into individual words. */
+  static class ExtractWords extends DoFn<String, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] words = c.element().split("[^a-zA-Z']+");
+      for (String word : words) {
+        if (!word.isEmpty()) {
+          c.output(word);
+        }
+      }
+    }
+  }
+
+  /** A DoFn that uppercases a word. */
+  static class Uppercase extends DoFn<String, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().toUpperCase());
+    }
+  }
+
+  /**
+   * Converts strings into BigQuery rows.
+   */
+  static class StringToRowConverter extends DoFn<String, TableRow> {
+    /**
+     * In this example, put the whole string into single BigQuery field.
+     */
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(new TableRow().set("string_field", c.element()));
+    }
+
+    static TableSchema getSchema() {
+      return new TableSchema().setFields(new ArrayList<TableFieldSchema>() {
+            // Compose the list of TableFieldSchema from tableSchema.
+            {
+              add(new TableFieldSchema().setName("string_field").setType("STRING"));
+            }
+      });
+    }
+  }
+
+  /**
+   * Options supported by {@link StreamingWordExtract}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private interface StreamingWordExtractOptions
+      extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
+    @Description("Input file to inject to Pub/Sub topic")
+    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
+    String getInputFile();
+    void setInputFile(String value);
+  }
+
+  /**
+   * Sets up and starts streaming pipeline.
+   *
+   * @throws IOException if there is a problem setting up resources
+   */
+  public static void main(String[] args) throws IOException {
+    StreamingWordExtractOptions options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(StreamingWordExtractOptions.class);
+    options.setStreaming(true);
+    // In order to cancel the pipelines automatically,
+    // {@literal DataflowPipelineRunner} is forced to be used.
+    options.setRunner(DataflowPipelineRunner.class);
+
+    options.setBigQuerySchema(StringToRowConverter.getSchema());
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
+    dataflowUtils.setup();
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    String tableSpec = new StringBuilder()
+        .append(options.getProject()).append(":")
+        .append(options.getBigQueryDataset()).append(".")
+        .append(options.getBigQueryTable())
+        .toString();
+    pipeline
+        .apply(PubsubIO.Read.topic(options.getPubsubTopic()))
+        .apply(ParDo.of(new ExtractWords()))
+        .apply(ParDo.of(new Uppercase()))
+        .apply(ParDo.of(new StringToRowConverter()))
+        .apply(BigQueryIO.Write.to(tableSpec)
+            .withSchema(StringToRowConverter.getSchema()));
+
+    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.waitToFinish(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..79ce823
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -0,0 +1,432 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.GcsOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.Keys;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.transforms.Values;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.WithKeys;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * An example that computes a basic TF-IDF search table for a directory or GCS prefix.
+ *
+ * <p>Concepts: joining data; side inputs; logging
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }</pre>
+ * and a local output file or output prefix on GCS:
+ * <pre>{@code
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * and an output prefix on GCS:
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ * }</pre>
+ *
+ * <p>The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with
+ * {@code --input}.
+ */
+public class TfIdf {
+  /**
+   * Options supported by {@link TfIdf}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Path to the directory or GCS prefix containing files to read from")
+    @Default.String("gs://dataflow-samples/shakespeare/")
+    String getInput();
+    void setInput(String value);
+
+    @Description("Prefix of output URI to write to")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  /**
+   * Lists documents contained beneath the {@code options.input} prefix/directory.
+   */
+  public static Set<URI> listInputDocuments(Options options)
+      throws URISyntaxException, IOException {
+    URI baseUri = new URI(options.getInput());
+
+    // List all documents in the directory or GCS prefix.
+    URI absoluteUri;
+    if (baseUri.getScheme() != null) {
+      absoluteUri = baseUri;
+    } else {
+      absoluteUri = new URI(
+          "file",
+          baseUri.getAuthority(),
+          baseUri.getPath(),
+          baseUri.getQuery(),
+          baseUri.getFragment());
+    }
+
+    Set<URI> uris = new HashSet<>();
+    if (absoluteUri.getScheme().equals("file")) {
+      File directory = new File(absoluteUri);
+      for (String entry : directory.list()) {
+        File path = new File(directory, entry);
+        uris.add(path.toURI());
+      }
+    } else if (absoluteUri.getScheme().equals("gs")) {
+      GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
+      URI gcsUriGlob = new URI(
+          absoluteUri.getScheme(),
+          absoluteUri.getAuthority(),
+          absoluteUri.getPath() + "*",
+          absoluteUri.getQuery(),
+          absoluteUri.getFragment());
+      for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
+        uris.add(entry.toUri());
+      }
+    }
+
+    return uris;
+  }
+
+  /**
+   * Reads the documents at the provided uris and returns all lines
+   * from the documents tagged with which document they are from.
+   */
+  public static class ReadDocuments
+      extends PTransform<PInput, PCollection<KV<URI, String>>> {
+    private Iterable<URI> uris;
+
+    public ReadDocuments(Iterable<URI> uris) {
+      this.uris = uris;
+    }
+
+    @Override
+    public Coder<?> getDefaultOutputCoder() {
+      return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
+    }
+
+    @Override
+    public PCollection<KV<URI, String>> apply(PInput input) {
+      Pipeline pipeline = input.getPipeline();
+
+      // Create one TextIO.Read transform for each document
+      // and add its output to a PCollectionList
+      PCollectionList<KV<URI, String>> urisToLines =
+          PCollectionList.empty(pipeline);
+
+      // TextIO.Read supports:
+      //  - file: URIs and paths locally
+      //  - gs: URIs on the service
+      for (final URI uri : uris) {
+        String uriString;
+        if (uri.getScheme().equals("file")) {
+          uriString = new File(uri).getPath();
+        } else {
+          uriString = uri.toString();
+        }
+
+        PCollection<KV<URI, String>> oneUriToLines = pipeline
+            .apply(TextIO.Read.from(uriString)
+                .named("TextIO.Read(" + uriString + ")"))
+            .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
+
+        urisToLines = urisToLines.and(oneUriToLines);
+      }
+
+      return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
+    }
+  }
+
+  /**
+   * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
+   * where the key is the document's URI and the value is a piece
+   * of the document's content. The output is mapping from terms to
+   * scores for each document URI.
+   */
+  public static class ComputeTfIdf
+      extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
+    public ComputeTfIdf() { }
+
+    @Override
+    public PCollection<KV<String, KV<URI, Double>>> apply(
+      PCollection<KV<URI, String>> uriToContent) {
+
+      // Compute the total number of documents, and
+      // prepare this singleton PCollectionView for
+      // use as a side input.
+      final PCollectionView<Long> totalDocuments =
+          uriToContent
+          .apply("GetURIs", Keys.<URI>create())
+          .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
+          .apply(Count.<URI>globally())
+          .apply(View.<Long>asSingleton());
+
+      // Create a collection of pairs mapping a URI to each
+      // of the words in the document associated with that that URI.
+      PCollection<KV<URI, String>> uriToWords = uriToContent
+          .apply(ParDo.named("SplitWords").of(
+              new DoFn<KV<URI, String>, KV<URI, String>>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  URI uri = c.element().getKey();
+                  String line = c.element().getValue();
+                  for (String word : line.split("\\W+")) {
+                    // Log INFO messages when the word “love” is found.
+                    if (word.toLowerCase().equals("love")) {
+                      LOG.info("Found {}", word.toLowerCase());
+                    }
+
+                    if (!word.isEmpty()) {
+                      c.output(KV.of(uri, word.toLowerCase()));
+                    }
+                  }
+                }
+              }));
+
+      // Compute a mapping from each word to the total
+      // number of documents in which it appears.
+      PCollection<KV<String, Long>> wordToDocCount = uriToWords
+          .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
+          .apply(Values.<String>create())
+          .apply("CountDocs", Count.<String>perElement());
+
+      // Compute a mapping from each URI to the total
+      // number of words in the document associated with that URI.
+      PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
+          .apply("GetURIs2", Keys.<URI>create())
+          .apply("CountWords", Count.<URI>perElement());
+
+      // Count, for each (URI, word) pair, the number of
+      // occurrences of that word in the document associated
+      // with the URI.
+      PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
+          .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
+
+      // Adjust the above collection to a mapping from
+      // (URI, word) pairs to counts into an isomorphic mapping
+      // from URI to (word, count) pairs, to prepare for a join
+      // by the URI key.
+      PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
+          .apply(ParDo.named("ShiftKeys").of(
+              new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  URI uri = c.element().getKey().getKey();
+                  String word = c.element().getKey().getValue();
+                  Long occurrences = c.element().getValue();
+                  c.output(KV.of(uri, KV.of(word, occurrences)));
+                }
+              }));
+
+      // Prepare to join the mapping of URI to (word, count) pairs with
+      // the mapping of URI to total word counts, by associating
+      // each of the input PCollection<KV<URI, ...>> with
+      // a tuple tag. Each input must have the same key type, URI
+      // in this case. The type parameter of the tuple tag matches
+      // the types of the values for each collection.
+      final TupleTag<Long> wordTotalsTag = new TupleTag<Long>();
+      final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<KV<String, Long>>();
+      KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
+          .of(wordTotalsTag, uriToWordTotal)
+          .and(wordCountsTag, uriToWordAndCount);
+
+      // Perform a CoGroupByKey (a sort of pre-join) on the prepared
+      // inputs. This yields a mapping from URI to a CoGbkResult
+      // (CoGroupByKey Result). The CoGbkResult is a mapping
+      // from the above tuple tags to the values in each input
+      // associated with a particular URI. In this case, each
+      // KV<URI, CoGbkResult> group a URI with the total number of
+      // words in that document as well as all the (word, count)
+      // pairs for particular words.
+      PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
+          .apply("CoGroupByUri", CoGroupByKey.<URI>create());
+
+      // Compute a mapping from each word to a (URI, term frequency)
+      // pair for each URI. A word's term frequency for a document
+      // is simply the number of times that word occurs in the document
+      // divided by the total number of words in the document.
+      PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
+          .apply(ParDo.named("ComputeTermFrequencies").of(
+              new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  URI uri = c.element().getKey();
+                  Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
+
+                  for (KV<String, Long> wordAndCount
+                           : c.element().getValue().getAll(wordCountsTag)) {
+                    String word = wordAndCount.getKey();
+                    Long wordCount = wordAndCount.getValue();
+                    Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
+                    c.output(KV.of(word, KV.of(uri, termFrequency)));
+                  }
+                }
+              }));
+
+      // Compute a mapping from each word to its document frequency.
+      // A word's document frequency in a corpus is the number of
+      // documents in which the word appears divided by the total
+      // number of documents in the corpus. Note how the total number of
+      // documents is passed as a side input; the same value is
+      // presented to each invocation of the DoFn.
+      PCollection<KV<String, Double>> wordToDf = wordToDocCount
+          .apply(ParDo
+              .named("ComputeDocFrequencies")
+              .withSideInputs(totalDocuments)
+              .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  String word = c.element().getKey();
+                  Long documentCount = c.element().getValue();
+                  Long documentTotal = c.sideInput(totalDocuments);
+                  Double documentFrequency = documentCount.doubleValue()
+                      / documentTotal.doubleValue();
+
+                  c.output(KV.of(word, documentFrequency));
+                }
+              }));
+
+      // Join the term frequency and document frequency
+      // collections, each keyed on the word.
+      final TupleTag<KV<URI, Double>> tfTag = new TupleTag<KV<URI, Double>>();
+      final TupleTag<Double> dfTag = new TupleTag<Double>();
+      PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
+          .of(tfTag, wordToUriAndTf)
+          .and(dfTag, wordToDf)
+          .apply(CoGroupByKey.<String>create());
+
+      // Compute a mapping from each word to a (URI, TF-IDF) score
+      // for each URI. There are a variety of definitions of TF-IDF
+      // ("term frequency - inverse document frequency") score;
+      // here we use a basic version that is the term frequency
+      // divided by the log of the document frequency.
+      PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = wordToUriAndTfAndDf
+          .apply(ParDo.named("ComputeTfIdf").of(
+              new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  String word = c.element().getKey();
+                  Double df = c.element().getValue().getOnly(dfTag);
+
+                  for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
+                    URI uri = uriAndTf.getKey();
+                    Double tf = uriAndTf.getValue();
+                    Double tfIdf = tf * Math.log(1 / df);
+                    c.output(KV.of(word, KV.of(uri, tfIdf)));
+                  }
+                }
+              }));
+
+      return wordToUriAndTfIdf;
+    }
+
+    // Instantiate Logger.
+    // It is suggested that the user specify the class name of the containing class
+    // (in this case ComputeTfIdf).
+    private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
+  }
+
+  /**
+   * A {@link PTransform} to write, in CSV format, a mapping from term and URI
+   * to score.
+   */
+  public static class WriteTfIdf
+      extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
+    private String output;
+
+    public WriteTfIdf(String output) {
+      this.output = output;
+    }
+
+    @Override
+    public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
+      return wordToUriAndTfIdf
+          .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
+            @Override
+            public void processElement(ProcessContext c) {
+              c.output(String.format("%s,\t%s,\t%f",
+                  c.element().getKey(),
+                  c.element().getValue().getKey(),
+                  c.element().getValue().getValue()));
+            }
+          }))
+          .apply(TextIO.Write
+              .to(output)
+              .withSuffix(".csv"));
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline pipeline = Pipeline.create(options);
+    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+    pipeline
+        .apply(new ReadDocuments(listInputDocuments(options)))
+        .apply(new ComputeTfIdf())
+        .apply(new WriteTfIdf(options.getOutput()));
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..f1d8d1a
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -0,0 +1,224 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableComparator;
+import com.google.cloud.dataflow.sdk.transforms.Top;
+import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.List;
+
+/**
+ * An example that reads Wikipedia edit data from Cloud Storage and computes the user with
+ * the longest string of edits separated by no more than an hour within each month.
+ *
+ * <p>Concepts: Using Windowing to perform time-based aggregations of data.
+ *
+ * <p>It is not recommended to execute this pipeline locally, given the size of the default input
+ * data.
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and an output prefix on GCS:
+ * <pre>{@code
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ * }</pre>
+ *
+ * <p>The default input is {@code gs://dataflow-samples/wikipedia_edits/*.json} and can be
+ * overridden with {@code --input}.
+ *
+ * <p>The input for this example is large enough that it's a good place to enable (experimental)
+ * autoscaling:
+ * <pre>{@code
+ *   --autoscalingAlgorithm=BASIC
+ *   --maxNumWorkers=20
+ * }
+ * </pre>
+ * This will automatically scale the number of workers up over time until the job completes.
+ */
+public class TopWikipediaSessions {
+  private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json";
+
+  /**
+   * Extracts user and timestamp from a TableRow representing a Wikipedia edit.
+   */
+  static class ExtractUserAndTimestamp extends DoFn<TableRow, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = c.element();
+      int timestamp = (Integer) row.get("timestamp");
+      String userName = (String) row.get("contributor_username");
+      if (userName != null) {
+        // Sets the implicit timestamp field to be used in windowing.
+        c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+      }
+    }
+  }
+
+  /**
+   * Computes the number of edits in each user session.  A session is defined as
+   * a string of edits where each is separated from the next by less than an hour.
+   */
+  static class ComputeSessions
+      extends PTransform<PCollection<String>, PCollection<KV<String, Long>>> {
+    @Override
+    public PCollection<KV<String, Long>> apply(PCollection<String> actions) {
+      return actions
+          .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardHours(1))))
+
+          .apply(Count.<String>perElement());
+    }
+  }
+
+  /**
+   * Computes the longest session ending in each month.
+   */
+  private static class TopPerMonth
+      extends PTransform<PCollection<KV<String, Long>>, PCollection<List<KV<String, Long>>>> {
+    @Override
+    public PCollection<List<KV<String, Long>>> apply(PCollection<KV<String, Long>> sessions) {
+      return sessions
+        .apply(Window.<KV<String, Long>>into(CalendarWindows.months(1)))
+
+          .apply(Top.of(1, new SerializableComparator<KV<String, Long>>() {
+                    @Override
+                    public int compare(KV<String, Long> o1, KV<String, Long> o2) {
+                      return Long.compare(o1.getValue(), o2.getValue());
+                    }
+                  }).withoutDefaults());
+    }
+  }
+
+  static class SessionsToStringsDoFn extends DoFn<KV<String, Long>, KV<String, Long>>
+      implements RequiresWindowAccess {
+
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(KV.of(
+          c.element().getKey() + " : " + c.window(), c.element().getValue()));
+    }
+  }
+
+  static class FormatOutputDoFn extends DoFn<List<KV<String, Long>>, String>
+      implements RequiresWindowAccess {
+    @Override
+    public void processElement(ProcessContext c) {
+      for (KV<String, Long> item : c.element()) {
+        String session = item.getKey();
+        long count = item.getValue();
+        c.output(session + " : " + count + " : " + ((IntervalWindow) c.window()).start());
+      }
+    }
+  }
+
+  static class ComputeTopSessions extends PTransform<PCollection<TableRow>, PCollection<String>> {
+
+    private final double samplingThreshold;
+
+    public ComputeTopSessions(double samplingThreshold) {
+      this.samplingThreshold = samplingThreshold;
+    }
+
+    @Override
+    public PCollection<String> apply(PCollection<TableRow> input) {
+      return input
+          .apply(ParDo.of(new ExtractUserAndTimestamp()))
+
+          .apply(ParDo.named("SampleUsers").of(
+              new DoFn<String, String>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) {
+                    c.output(c.element());
+                  }
+                }
+              }))
+
+          .apply(new ComputeSessions())
+
+          .apply(ParDo.named("SessionsToStrings").of(new SessionsToStringsDoFn()))
+          .apply(new TopPerMonth())
+          .apply(ParDo.named("FormatOutput").of(new FormatOutputDoFn()));
+    }
+  }
+
+  /**
+   * Options supported by this class.
+   *
+   * <p>Inherits standard Dataflow configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description(
+      "Input specified as a GCS path containing a BigQuery table exported as json")
+    @Default.String(EXPORTED_WIKI_TABLE)
+    String getInput();
+    void setInput(String value);
+
+    @Description("File to output results to")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  public static void main(String[] args) {
+    Options options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(Options.class);
+    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+
+    Pipeline p = Pipeline.create(dataflowOptions);
+
+    double samplingThreshold = 0.1;
+
+    p.apply(TextIO.Read
+        .from(options.getInput())
+        .withCoder(TableRowJsonCoder.of()))
+     .apply(new ComputeTopSessions(samplingThreshold))
+     .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput()));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..2c3c857
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
@@ -0,0 +1,426 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+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.cloud.dataflow.examples.common.DataflowExampleOptions;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Strings;
+
+import org.apache.avro.reflect.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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>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>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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
+ * and then exits.
+ */
+public class TrafficMaxLaneFlow {
+
+  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
+  private static final Integer VALID_INPUTS = 4999;
+
+  static final int WINDOW_DURATION = 60;  // Default sliding window duration in minutes
+  static final int WINDOW_SLIDE_EVERY = 5;  // Default window 'slide every' setting in minutes
+
+  /**
+   * This class holds information about each lane in a station reading, along with some general
+   * information from the reading.
+   */
+  @DefaultCoder(AvroCoder.class)
+  static class LaneInfo {
+    @Nullable String stationId;
+    @Nullable String lane;
+    @Nullable String direction;
+    @Nullable String freeway;
+    @Nullable String recordedTimestamp;
+    @Nullable Integer laneFlow;
+    @Nullable Integer totalFlow;
+    @Nullable Double laneAO;
+    @Nullable Double laneAS;
+
+    public LaneInfo() {}
+
+    public LaneInfo(String stationId, String lane, String direction, String freeway,
+        String timestamp, Integer laneFlow, Double laneAO,
+        Double laneAS, Integer totalFlow) {
+      this.stationId = stationId;
+      this.lane = lane;
+      this.direction = direction;
+      this.freeway = freeway;
+      this.recordedTimestamp = timestamp;
+      this.laneFlow = laneFlow;
+      this.laneAO = laneAO;
+      this.laneAS = laneAS;
+      this.totalFlow = totalFlow;
+    }
+
+    public String getStationId() {
+      return this.stationId;
+    }
+    public String getLane() {
+      return this.lane;
+    }
+    public String getDirection() {
+      return this.direction;
+    }
+    public String getFreeway() {
+      return this.freeway;
+    }
+    public String getRecordedTimestamp() {
+      return this.recordedTimestamp;
+    }
+    public Integer getLaneFlow() {
+      return this.laneFlow;
+    }
+    public Double getLaneAO() {
+      return this.laneAO;
+    }
+    public Double getLaneAS() {
+      return this.laneAS;
+    }
+    public Integer getTotalFlow() {
+      return this.totalFlow;
+    }
+  }
+
+  /**
+   * Extract the timestamp field from the input string, and use it as the element timestamp.
+   */
+  static class ExtractTimestamps extends DoFn<String, String> {
+    private static final DateTimeFormatter dateTimeFormat =
+        DateTimeFormat.forPattern("MM/dd/yyyy HH:mm:ss");
+
+    @Override
+    public void processElement(DoFn<String, String>.ProcessContext c) throws Exception {
+      String[] items = c.element().split(",");
+      if (items.length > 0) {
+        try {
+          String timestamp = items[0];
+          c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp)));
+        } catch (IllegalArgumentException e) {
+          // Skip the invalid input.
+        }
+      }
+    }
+  }
+
+  /**
+   * Extract flow information for each of the 8 lanes in a reading, and output as separate tuples.
+   * This will let us determine which lane has the max flow for that station over the span of the
+   * window, and output not only the max flow from that calculation, but other associated
+   * information. The number of lanes for which data is present depends upon which freeway the data
+   * point comes from.
+   */
+  static class ExtractFlowInfoFn extends DoFn<String, KV<String, LaneInfo>> {
+
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] items = c.element().split(",");
+      if (items.length < 48) {
+        // Skip the invalid input.
+        return;
+      }
+      // extract the sensor information for the lanes from the input string fields.
+      String timestamp = items[0];
+      String stationId = items[1];
+      String freeway = items[2];
+      String direction = items[3];
+      Integer totalFlow = tryIntParse(items[7]);
+      for (int i = 1; i <= 8; ++i) {
+        Integer laneFlow = tryIntParse(items[6 + 5 * i]);
+        Double laneAvgOccupancy = tryDoubleParse(items[7 + 5 * i]);
+        Double laneAvgSpeed = tryDoubleParse(items[8 + 5 * i]);
+        if (laneFlow == null || laneAvgOccupancy == null || laneAvgSpeed == null) {
+          return;
+        }
+        LaneInfo laneInfo = new LaneInfo(stationId, "lane" + i, direction, freeway, timestamp,
+            laneFlow, laneAvgOccupancy, laneAvgSpeed, totalFlow);
+        c.output(KV.of(stationId, laneInfo));
+      }
+    }
+  }
+
+  /**
+   * A custom 'combine function' used with the Combine.perKey transform. Used to find the max lane
+   * flow over all the data points in the Window. Extracts the lane flow from the input string and
+   * determines whether it's the max seen so far. We're using a custom combiner instead of the Max
+   * transform because we want to retain the additional information we've associated with the flow
+   * value.
+   */
+  public static class MaxFlow implements SerializableFunction<Iterable<LaneInfo>, LaneInfo> {
+    @Override
+    public LaneInfo apply(Iterable<LaneInfo> input) {
+      Integer max = 0;
+      LaneInfo maxInfo = new LaneInfo();
+      for (LaneInfo item : input) {
+        Integer flow = item.getLaneFlow();
+        if (flow != null && (flow >= max)) {
+          max = flow;
+          maxInfo = item;
+        }
+      }
+      return maxInfo;
+    }
+  }
+
+  /**
+   * Format the results of the Max Lane flow calculation to a TableRow, to save to BigQuery.
+   * Add the timestamp from the window context.
+   */
+  static class FormatMaxesFn extends DoFn<KV<String, LaneInfo>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+
+      LaneInfo laneInfo = c.element().getValue();
+      TableRow row = new TableRow()
+          .set("station_id", c.element().getKey())
+          .set("direction", laneInfo.getDirection())
+          .set("freeway", laneInfo.getFreeway())
+          .set("lane_max_flow", laneInfo.getLaneFlow())
+          .set("lane", laneInfo.getLane())
+          .set("avg_occ", laneInfo.getLaneAO())
+          .set("avg_speed", laneInfo.getLaneAS())
+          .set("total_flow", laneInfo.getTotalFlow())
+          .set("recorded_timestamp", laneInfo.getRecordedTimestamp())
+          .set("window_timestamp", c.timestamp().toString());
+      c.output(row);
+    }
+
+    /** Defines the BigQuery schema used for the output. */
+    static TableSchema getSchema() {
+      List<TableFieldSchema> fields = new ArrayList<>();
+      fields.add(new TableFieldSchema().setName("station_id").setType("STRING"));
+      fields.add(new TableFieldSchema().setName("direction").setType("STRING"));
+      fields.add(new TableFieldSchema().setName("freeway").setType("STRING"));
+      fields.add(new TableFieldSchema().setName("lane_max_flow").setType("INTEGER"));
+      fields.add(new TableFieldSchema().setName("lane").setType("STRING"));
+      fields.add(new TableFieldSchema().setName("avg_occ").setType("FLOAT"));
+      fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT"));
+      fields.add(new TableFieldSchema().setName("total_flow").setType("INTEGER"));
+      fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
+      fields.add(new TableFieldSchema().setName("recorded_timestamp").setType("STRING"));
+      TableSchema schema = new TableSchema().setFields(fields);
+      return schema;
+    }
+  }
+
+  /**
+   * This PTransform extracts lane info, calculates the max lane flow found for a given station (for
+   * the current Window) using a custom 'combiner', and formats the results for BigQuery.
+   */
+  static class MaxLaneFlow
+      extends PTransform<PCollection<KV<String, LaneInfo>>, PCollection<TableRow>> {
+    @Override
+    public PCollection<TableRow> apply(PCollection<KV<String, LaneInfo>> flowInfo) {
+      // stationId, LaneInfo => stationId + max lane flow info
+      PCollection<KV<String, LaneInfo>> flowMaxes =
+          flowInfo.apply(Combine.<String, LaneInfo>perKey(
+              new MaxFlow()));
+
+      // <stationId, max lane flow info>... => row...
+      PCollection<TableRow> results = flowMaxes.apply(
+          ParDo.of(new FormatMaxesFn()));
+
+      return results;
+    }
+  }
+
+  static class ReadFileAndExtractTimestamps extends PTransform<PBegin, PCollection<String>> {
+    private final String inputFile;
+
+    public ReadFileAndExtractTimestamps(String inputFile) {
+      this.inputFile = inputFile;
+    }
+
+    @Override
+    public PCollection<String> apply(PBegin begin) {
+      return begin
+          .apply(TextIO.Read.from(inputFile))
+          .apply(ParDo.of(new ExtractTimestamps()));
+    }
+  }
+
+  /**
+    * Options supported by {@link TrafficMaxLaneFlow}.
+    *
+    * <p>Inherits standard configuration options.
+    */
+  private interface TrafficMaxLaneFlowOptions extends DataflowExampleOptions,
+      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
+        @Description("Input file to inject to Pub/Sub topic")
+    @Default.String("gs://dataflow-samples/traffic_sensor/"
+        + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
+    String getInputFile();
+    void setInputFile(String value);
+
+    @Description("Numeric value of sliding window duration, in minutes")
+    @Default.Integer(WINDOW_DURATION)
+    Integer getWindowDuration();
+    void setWindowDuration(Integer value);
+
+    @Description("Numeric value of window 'slide every' setting, in minutes")
+    @Default.Integer(WINDOW_SLIDE_EVERY)
+    Integer getWindowSlideEvery();
+    void setWindowSlideEvery(Integer value);
+
+    @Description("Whether to run the pipeline with unbounded input")
+    @Default.Boolean(false)
+    boolean isUnbounded();
+    void setUnbounded(boolean value);
+  }
+
+  /**
+   * Sets up and starts streaming pipeline.
+   *
+   * @throws IOException if there is a problem setting up resources
+   */
+  public static void main(String[] args) throws IOException {
+    TrafficMaxLaneFlowOptions options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(TrafficMaxLaneFlowOptions.class);
+    options.setBigQuerySchema(FormatMaxesFn.getSchema());
+    // Using DataflowExampleUtils to set up required resources.
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options, options.isUnbounded());
+
+    Pipeline pipeline = Pipeline.create(options);
+    TableReference tableRef = new TableReference();
+    tableRef.setProjectId(options.getProject());
+    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
+        // 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.
+        .apply(Window.<KV<String, LaneInfo>>into(SlidingWindows.of(
+            Duration.standardMinutes(options.getWindowDuration())).
+            every(Duration.standardMinutes(options.getWindowSlideEvery()))))
+        .apply(new MaxLaneFlow())
+        .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();
+
+    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    dataflowUtils.waitToFinish(result);
+  }
+
+  private static Integer tryIntParse(String number) {
+    try {
+      return Integer.parseInt(number);
+    } catch (NumberFormatException e) {
+      return null;
+    }
+  }
+
+  private static Double tryDoubleParse(String number) {
+    try {
+      return Double.parseDouble(number);
+    } catch (NumberFormatException e) {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..b1c72e6
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
@@ -0,0 +1,460 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+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.cloud.dataflow.examples.common.DataflowExampleOptions;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+
+import org.apache.avro.reflect.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.List;
+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>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>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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
+ * and then exits.
+ */
+
+public class TrafficRoutes {
+
+  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
+  private static final Integer VALID_INPUTS = 4999;
+
+  // Instantiate some small predefined San Diego routes to analyze
+  static Map<String, String> sdStations = buildStationInfo();
+  static final int WINDOW_DURATION = 3;  // Default sliding window duration in minutes
+  static final int WINDOW_SLIDE_EVERY = 1;  // Default window 'slide every' setting in minutes
+
+  /**
+   * This class holds information about a station reading's average speed.
+   */
+  @DefaultCoder(AvroCoder.class)
+  static class StationSpeed implements Comparable<StationSpeed> {
+    @Nullable String stationId;
+    @Nullable Double avgSpeed;
+    @Nullable Long timestamp;
+
+    public StationSpeed() {}
+
+    public StationSpeed(String stationId, Double avgSpeed, Long timestamp) {
+      this.stationId = stationId;
+      this.avgSpeed = avgSpeed;
+      this.timestamp = timestamp;
+    }
+
+    public String getStationId() {
+      return this.stationId;
+    }
+    public Double getAvgSpeed() {
+      return this.avgSpeed;
+    }
+
+    @Override
+    public int compareTo(StationSpeed other) {
+      return Long.compare(this.timestamp, other.timestamp);
+    }
+  }
+
+  /**
+   * This class holds information about a route's speed/slowdown.
+   */
+  @DefaultCoder(AvroCoder.class)
+  static class RouteInfo {
+    @Nullable String route;
+    @Nullable Double avgSpeed;
+    @Nullable Boolean slowdownEvent;
+
+
+    public RouteInfo() {}
+
+    public RouteInfo(String route, Double avgSpeed, Boolean slowdownEvent) {
+      this.route = route;
+      this.avgSpeed = avgSpeed;
+      this.slowdownEvent = slowdownEvent;
+    }
+
+    public String getRoute() {
+      return this.route;
+    }
+    public Double getAvgSpeed() {
+      return this.avgSpeed;
+    }
+    public Boolean getSlowdownEvent() {
+      return this.slowdownEvent;
+    }
+  }
+
+  /**
+   * Extract the timestamp field from the input string, and use it as the element timestamp.
+   */
+  static class ExtractTimestamps extends DoFn<String, String> {
+    private static final DateTimeFormatter dateTimeFormat =
+        DateTimeFormat.forPattern("MM/dd/yyyy HH:mm:ss");
+
+    @Override
+    public void processElement(DoFn<String, String>.ProcessContext c) throws Exception {
+      String[] items = c.element().split(",");
+      String timestamp = tryParseTimestamp(items);
+      if (timestamp != null) {
+        try {
+          c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp)));
+        } catch (IllegalArgumentException e) {
+          // Skip the invalid input.
+        }
+      }
+    }
+  }
+
+  /**
+   * Filter out readings for the stations along predefined 'routes', and output
+   * (station, speed info) keyed on route.
+   */
+  static class ExtractStationSpeedFn extends DoFn<String, KV<String, StationSpeed>> {
+
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] items = c.element().split(",");
+      String stationType = tryParseStationType(items);
+      // For this analysis, use only 'main line' station types
+      if (stationType != null && stationType.equals("ML")) {
+        Double avgSpeed = tryParseAvgSpeed(items);
+        String stationId = tryParseStationId(items);
+        // For this simple example, filter out everything but some hardwired routes.
+        if (avgSpeed != null && stationId != null && sdStations.containsKey(stationId)) {
+          StationSpeed stationSpeed =
+              new StationSpeed(stationId, avgSpeed, c.timestamp().getMillis());
+          // The tuple key is the 'route' name stored in the 'sdStations' hash.
+          KV<String, StationSpeed> outputValue = KV.of(sdStations.get(stationId), stationSpeed);
+          c.output(outputValue);
+        }
+      }
+    }
+  }
+
+  /**
+   * For a given route, track average speed for the window. Calculate whether
+   * traffic is currently slowing down, via a predefined threshold. If a supermajority of
+   * speeds in this sliding window are less than the previous reading we call this a 'slowdown'.
+   * Note: these calculations are for example purposes only, and are unrealistic and oversimplified.
+   */
+  static class GatherStats
+      extends DoFn<KV<String, Iterable<StationSpeed>>, KV<String, RouteInfo>> {
+    @Override
+    public void processElement(ProcessContext c) throws IOException {
+      String route = c.element().getKey();
+      double speedSum = 0.0;
+      int speedCount = 0;
+      int speedups = 0;
+      int slowdowns = 0;
+      List<StationSpeed> infoList = Lists.newArrayList(c.element().getValue());
+      // StationSpeeds sort by embedded timestamp.
+      Collections.sort(infoList);
+      Map<String, Double> prevSpeeds = new HashMap<>();
+      // For all stations in the route, sum (non-null) speeds. Keep a count of the non-null speeds.
+      for (StationSpeed item : infoList) {
+        Double speed = item.getAvgSpeed();
+        if (speed != null) {
+          speedSum += speed;
+          speedCount++;
+          Double lastSpeed = prevSpeeds.get(item.getStationId());
+          if (lastSpeed != null) {
+            if (lastSpeed < speed) {
+              speedups += 1;
+            } else {
+              slowdowns += 1;
+            }
+          }
+          prevSpeeds.put(item.getStationId(), speed);
+        }
+      }
+      if (speedCount == 0) {
+        // No average to compute.
+        return;
+      }
+      double speedAvg = speedSum / speedCount;
+      boolean slowdownEvent = slowdowns >= 2 * speedups;
+      RouteInfo routeInfo = new RouteInfo(route, speedAvg, slowdownEvent);
+      c.output(KV.of(route, routeInfo));
+    }
+  }
+
+  /**
+   * Format the results of the slowdown calculations to a TableRow, to save to BigQuery.
+   */
+  static class FormatStatsFn extends DoFn<KV<String, RouteInfo>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      RouteInfo routeInfo = c.element().getValue();
+      TableRow row = new TableRow()
+          .set("avg_speed", routeInfo.getAvgSpeed())
+          .set("slowdown_event", routeInfo.getSlowdownEvent())
+          .set("route", c.element().getKey())
+          .set("window_timestamp", c.timestamp().toString());
+      c.output(row);
+    }
+
+    /**
+     * Defines the BigQuery schema used for the output.
+     */
+    static TableSchema getSchema() {
+      List<TableFieldSchema> fields = new ArrayList<>();
+      fields.add(new TableFieldSchema().setName("route").setType("STRING"));
+      fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT"));
+      fields.add(new TableFieldSchema().setName("slowdown_event").setType("BOOLEAN"));
+      fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
+      TableSchema schema = new TableSchema().setFields(fields);
+      return schema;
+    }
+  }
+
+  /**
+   * This PTransform extracts speed info from traffic station readings.
+   * It groups the readings by 'route' and analyzes traffic slowdown for that route.
+   * Lastly, it formats the results for BigQuery.
+   */
+  static class TrackSpeed extends
+      PTransform<PCollection<KV<String, StationSpeed>>, PCollection<TableRow>> {
+    @Override
+    public PCollection<TableRow> apply(PCollection<KV<String, StationSpeed>> stationSpeed) {
+      // Apply a GroupByKey transform to collect a list of all station
+      // readings for a given route.
+      PCollection<KV<String, Iterable<StationSpeed>>> timeGroup = stationSpeed.apply(
+        GroupByKey.<String, StationSpeed>create());
+
+      // Analyze 'slowdown' over the route readings.
+      PCollection<KV<String, RouteInfo>> stats = timeGroup.apply(ParDo.of(new GatherStats()));
+
+      // Format the results for writing to BigQuery
+      PCollection<TableRow> results = stats.apply(
+          ParDo.of(new FormatStatsFn()));
+
+      return results;
+    }
+  }
+
+  static class ReadFileAndExtractTimestamps extends PTransform<PBegin, PCollection<String>> {
+    private final String inputFile;
+
+    public ReadFileAndExtractTimestamps(String inputFile) {
+      this.inputFile = inputFile;
+    }
+
+    @Override
+    public PCollection<String> apply(PBegin begin) {
+      return begin
+          .apply(TextIO.Read.from(inputFile))
+          .apply(ParDo.of(new ExtractTimestamps()));
+    }
+  }
+
+  /**
+  * Options supported by {@link TrafficRoutes}.
+  *
+  * <p>Inherits standard configuration options.
+  */
+  private interface TrafficRoutesOptions extends DataflowExampleOptions,
+      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
+    @Description("Input file to inject to Pub/Sub topic")
+    @Default.String("gs://dataflow-samples/traffic_sensor/"
+        + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
+    String getInputFile();
+    void setInputFile(String value);
+
+    @Description("Numeric value of sliding window duration, in minutes")
+    @Default.Integer(WINDOW_DURATION)
+    Integer getWindowDuration();
+    void setWindowDuration(Integer value);
+
+    @Description("Numeric value of window 'slide every' setting, in minutes")
+    @Default.Integer(WINDOW_SLIDE_EVERY)
+    Integer getWindowSlideEvery();
+    void setWindowSlideEvery(Integer value);
+
+    @Description("Whether to run the pipeline with unbounded input")
+    @Default.Boolean(false)
+    boolean isUnbounded();
+    void setUnbounded(boolean value);
+  }
+
+  /**
+   * Sets up and starts streaming pipeline.
+   *
+   * @throws IOException if there is a problem setting up resources
+   */
+  public static void main(String[] args) throws IOException {
+    TrafficRoutesOptions options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(TrafficRoutesOptions.class);
+
+    options.setBigQuerySchema(FormatStatsFn.getSchema());
+    // Using DataflowExampleUtils to set up required resources.
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options, options.isUnbounded());
+
+    Pipeline pipeline = Pipeline.create(options);
+    TableReference tableRef = new TableReference();
+    tableRef.setProjectId(options.getProject());
+    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
+        // 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()))))
+        .apply(new TrackSpeed())
+        .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();
+
+    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    dataflowUtils.waitToFinish(result);
+  }
+
+  private static Double tryParseAvgSpeed(String[] inputItems) {
+    try {
+      return Double.parseDouble(tryParseString(inputItems, 9));
+    } catch (NumberFormatException e) {
+      return null;
+    } catch (NullPointerException e) {
+      return null;
+    }
+  }
+
+  private static String tryParseStationType(String[] inputItems) {
+    return tryParseString(inputItems, 4);
+  }
+
+  private static String tryParseStationId(String[] inputItems) {
+    return tryParseString(inputItems, 1);
+  }
+
+  private static String tryParseTimestamp(String[] inputItems) {
+    return tryParseString(inputItems, 0);
+  }
+
+  private static String tryParseString(String[] inputItems, int index) {
+    return inputItems.length >= index ? inputItems[index] : null;
+  }
+
+  /**
+   * Define some small hard-wired San Diego 'routes' to track based on sensor station ID.
+   */
+  private static Map<String, String> buildStationInfo() {
+    Map<String, String> stations = new Hashtable<String, String>();
+      stations.put("1108413", "SDRoute1"); // from freeway 805 S
+      stations.put("1108699", "SDRoute2"); // from freeway 78 E
+      stations.put("1108702", "SDRoute2");
+    return stations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
new file mode 100644
index 0000000..e5fd015
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
@@ -0,0 +1,180 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An example that reads the public samples of weather data from BigQuery, counts the number of
+ * tornadoes that occur in each month, and writes the results to BigQuery.
+ *
+ * <p>Concepts: Reading/writing BigQuery; counting a PCollection; user-defined PTransforms
+ *
+ * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
+ * table.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and the BigQuery table for the output, with the form
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
+ * and can be overridden with {@code --input}.
+ */
+public class BigQueryTornadoes {
+  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
+  private static final String WEATHER_SAMPLES_TABLE =
+      "clouddataflow-readonly:samples.weather_stations";
+
+  /**
+   * Examines each row in the input table. If a tornado was recorded
+   * in that sample, the month in which it occurred is output.
+   */
+  static class ExtractTornadoesFn extends DoFn<TableRow, Integer> {
+    @Override
+    public void processElement(ProcessContext c){
+      TableRow row = c.element();
+      if ((Boolean) row.get("tornado")) {
+        c.output(Integer.parseInt((String) row.get("month")));
+      }
+    }
+  }
+
+  /**
+   * Prepares the data for writing to BigQuery by building a TableRow object containing an
+   * integer representation of month and the number of tornadoes that occurred in each month.
+   */
+  static class FormatCountsFn extends DoFn<KV<Integer, Long>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = new TableRow()
+          .set("month", c.element().getKey())
+          .set("tornado_count", c.element().getValue());
+      c.output(row);
+    }
+  }
+
+  /**
+   * Takes rows from a table and generates a table of counts.
+   *
+   * <p>The input schema is described by
+   * https://developers.google.com/bigquery/docs/dataset-gsod .
+   * The output contains the total number of tornadoes found in each month in
+   * the following schema:
+   * <ul>
+   *   <li>month: integer</li>
+   *   <li>tornado_count: integer</li>
+   * </ul>
+   */
+  static class CountTornadoes
+      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
+    @Override
+    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
+
+      // row... => month...
+      PCollection<Integer> tornadoes = rows.apply(
+          ParDo.of(new ExtractTornadoesFn()));
+
+      // month... => <month,count>...
+      PCollection<KV<Integer, Long>> tornadoCounts =
+          tornadoes.apply(Count.<Integer>perElement());
+
+      // <month,count>... => row...
+      PCollection<TableRow> results = tornadoCounts.apply(
+          ParDo.of(new FormatCountsFn()));
+
+      return results;
+    }
+  }
+
+  /**
+   * Options supported by {@link BigQueryTornadoes}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Table to read from, specified as "
+        + "<project_id>:<dataset_id>.<table_id>")
+    @Default.String(WEATHER_SAMPLES_TABLE)
+    String getInput();
+    void setInput(String value);
+
+    @Description("BigQuery table to write to, specified as "
+        + "<project_id>:<dataset_id>.<table_id>. The dataset must already exist.")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  public static void main(String[] args) {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+    Pipeline p = Pipeline.create(options);
+
+    // Build the table schema for the output table.
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("tornado_count").setType("INTEGER"));
+    TableSchema schema = new TableSchema().setFields(fields);
+
+    p.apply(BigQueryIO.Read.from(options.getInput()))
+     .apply(new CountTornadoes())
+     .apply(BigQueryIO.Write
+        .to(options.getOutput())
+        .withSchema(schema)
+        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+
+    p.run();
+  }
+}


[14/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java
deleted file mode 100644
index deed9ab..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java
+++ /dev/null
@@ -1,1048 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.MoreObjects.firstNonNull;
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.client.util.Clock;
-import com.google.api.client.util.DateTime;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.AcknowledgeRequest;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.api.services.pubsub.model.PullRequest;
-import com.google.api.services.pubsub.model.PullResponse;
-import com.google.api.services.pubsub.model.ReceivedMessage;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.PubsubOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.annotation.Nullable;
-
-/**
- * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create
- * and consume unbounded {@link PCollection PCollections}.
- *
- * <h3>Permissions</h3>
- * <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding
- * {@link PipelineRunner PipelineRunners} for more details.
- */
-public class PubsubIO {
-  private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class);
-
-  /** The default {@link Coder} used to translate to/from Cloud Pub/Sub messages. */
-  public static final Coder<String> DEFAULT_PUBSUB_CODER = StringUtf8Coder.of();
-
-  /**
-   * Project IDs must contain 6-63 lowercase letters, digits, or dashes.
-   * IDs must start with a letter and may not end with a dash.
-   * This regex isn't exact - this allows for patterns that would be rejected by
-   * the service, but this is sufficient for basic parsing of table references.
-   */
-  private static final Pattern PROJECT_ID_REGEXP =
-      Pattern.compile("[a-z][-a-z0-9:.]{4,61}[a-z0-9]");
-
-  private static final Pattern SUBSCRIPTION_REGEXP =
-      Pattern.compile("projects/([^/]+)/subscriptions/(.+)");
-
-  private static final Pattern TOPIC_REGEXP = Pattern.compile("projects/([^/]+)/topics/(.+)");
-
-  private static final Pattern V1BETA1_SUBSCRIPTION_REGEXP =
-      Pattern.compile("/subscriptions/([^/]+)/(.+)");
-
-  private static final Pattern V1BETA1_TOPIC_REGEXP = Pattern.compile("/topics/([^/]+)/(.+)");
-
-  private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+");
-
-  private static final int PUBSUB_NAME_MIN_LENGTH = 3;
-  private static final int PUBSUB_NAME_MAX_LENGTH = 255;
-
-  private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/";
-  private static final String SUBSCRIPTION_STARTING_SIGNAL = "_starting_signal/";
-  private static final String TOPIC_DEV_NULL_TEST_NAME = "/topics/dev/null";
-
-  private static void validateProjectName(String project) {
-    Matcher match = PROJECT_ID_REGEXP.matcher(project);
-    if (!match.matches()) {
-      throw new IllegalArgumentException(
-          "Illegal project name specified in Pubsub subscription: " + project);
-    }
-  }
-
-  private static void validatePubsubName(String name) {
-    if (name.length() < PUBSUB_NAME_MIN_LENGTH) {
-      throw new IllegalArgumentException(
-          "Pubsub object name is shorter than 3 characters: " + name);
-    }
-    if (name.length() > PUBSUB_NAME_MAX_LENGTH) {
-      throw new IllegalArgumentException(
-          "Pubsub object name is longer than 255 characters: " + name);
-    }
-
-    if (name.startsWith("goog")) {
-      throw new IllegalArgumentException("Pubsub object name cannot start with goog: " + name);
-    }
-
-    Matcher match = PUBSUB_NAME_REGEXP.matcher(name);
-    if (!match.matches()) {
-      throw new IllegalArgumentException("Illegal Pubsub object name specified: " + name
-          + " Please see Javadoc for naming rules.");
-    }
-  }
-
-  /**
-   * Returns the {@link Instant} that corresponds to the timestamp in the supplied
-   * {@link PubsubMessage} under the specified {@code ink label}. See
-   * {@link PubsubIO.Read#timestampLabel(String)} for details about how these messages are
-   * parsed.
-   *
-   * <p>The {@link Clock} parameter is used to virtualize time for testing.
-   *
-   * @throws IllegalArgumentException if the timestamp label is provided, but there is no
-   *     corresponding attribute in the message or the value provided is not a valid timestamp
-   *     string.
-   * @see PubsubIO.Read#timestampLabel(String)
-   */
-  @VisibleForTesting
-  protected static Instant assignMessageTimestamp(
-      PubsubMessage message, @Nullable String label, Clock clock) {
-    if (label == null) {
-      return new Instant(clock.currentTimeMillis());
-    }
-
-    // Extract message attributes, defaulting to empty map if null.
-    Map<String, String> attributes = firstNonNull(
-        message.getAttributes(), ImmutableMap.<String, String>of());
-
-    String timestampStr = attributes.get(label);
-    checkArgument(timestampStr != null && !timestampStr.isEmpty(),
-        "PubSub message is missing a timestamp in label: %s", label);
-
-    long millisSinceEpoch;
-    try {
-      // Try parsing as milliseconds since epoch. Note there is no way to parse a string in
-      // RFC 3339 format here.
-      // Expected IllegalArgumentException if parsing fails; we use that to fall back to RFC 3339.
-      millisSinceEpoch = Long.parseLong(timestampStr);
-    } catch (IllegalArgumentException e) {
-      // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an IllegalArgumentException
-      // if parsing fails, and the caller should handle.
-      millisSinceEpoch = DateTime.parseRfc3339(timestampStr).getValue();
-    }
-    return new Instant(millisSinceEpoch);
-  }
-
-  /**
-   * Class representing a Cloud Pub/Sub Subscription.
-   */
-  public static class PubsubSubscription implements Serializable {
-    private enum Type { NORMAL, FAKE }
-
-    private final Type type;
-    private final String project;
-    private final String subscription;
-
-    private PubsubSubscription(Type type, String project, String subscription) {
-      this.type = type;
-      this.project = project;
-      this.subscription = subscription;
-    }
-
-    /**
-     * Creates a class representing a Pub/Sub subscription from the specified subscription path.
-     *
-     * <p>Cloud Pub/Sub subscription names should be of the form
-     * {@code projects/<project>/subscriptions/<subscription>}, where {@code <project>} is the name
-     * of the project the subscription belongs to. The {@code <subscription>} component must comply
-     * with the following requirements:
-     *
-     * <ul>
-     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
-     * ('.').</li>
-     * <li>Must be between 3 and 255 characters.</li>
-     * <li>Must begin with a letter.</li>
-     * <li>Must end with a letter or a number.</li>
-     * <li>Cannot begin with {@code 'goog'} prefix.</li>
-     * </ul>
-     */
-    public static PubsubSubscription fromPath(String path) {
-      if (path.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX)
-          || path.startsWith(SUBSCRIPTION_STARTING_SIGNAL)) {
-        return new PubsubSubscription(Type.FAKE, "", path);
-      }
-
-      String projectName, subscriptionName;
-
-      Matcher v1beta1Match = V1BETA1_SUBSCRIPTION_REGEXP.matcher(path);
-      if (v1beta1Match.matches()) {
-        LOG.warn("Saw subscription in v1beta1 format. Subscriptions should be in the format "
-            + "projects/<project_id>/subscriptions/<subscription_name>");
-        projectName = v1beta1Match.group(1);
-        subscriptionName = v1beta1Match.group(2);
-      } else {
-        Matcher match = SUBSCRIPTION_REGEXP.matcher(path);
-        if (!match.matches()) {
-          throw new IllegalArgumentException("Pubsub subscription is not in "
-              + "projects/<project_id>/subscriptions/<subscription_name> format: " + path);
-        }
-        projectName = match.group(1);
-        subscriptionName = match.group(2);
-      }
-
-      validateProjectName(projectName);
-      validatePubsubName(subscriptionName);
-      return new PubsubSubscription(Type.NORMAL, projectName, subscriptionName);
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * v1beta1 API.
-     *
-     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta1Path() {
-      if (type == Type.NORMAL) {
-        return "/subscriptions/" + project + "/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * v1beta2 API.
-     *
-     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta2Path() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/subscriptions/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-
-    /**
-     * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub
-     * API.
-     */
-    public String asPath() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/subscriptions/" + subscription;
-      } else {
-        return subscription;
-      }
-    }
-  }
-
-  /**
-   * Class representing a Cloud Pub/Sub Topic.
-   */
-  public static class PubsubTopic implements Serializable {
-    private enum Type { NORMAL, FAKE }
-
-    private final Type type;
-    private final String project;
-    private final String topic;
-
-    private PubsubTopic(Type type, String project, String topic) {
-      this.type = type;
-      this.project = project;
-      this.topic = topic;
-    }
-
-    /**
-     * Creates a class representing a Cloud Pub/Sub topic from the specified topic path.
-     *
-     * <p>Cloud Pub/Sub topic names should be of the form
-     * {@code /topics/<project>/<topic>}, where {@code <project>} is the name of
-     * the publishing project. The {@code <topic>} component must comply with
-     * the following requirements:
-     *
-     * <ul>
-     * <li>Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods
-     * ('.').</li>
-     * <li>Must be between 3 and 255 characters.</li>
-     * <li>Must begin with a letter.</li>
-     * <li>Must end with a letter or a number.</li>
-     * <li>Cannot begin with 'goog' prefix.</li>
-     * </ul>
-     */
-    public static PubsubTopic fromPath(String path) {
-      if (path.equals(TOPIC_DEV_NULL_TEST_NAME)) {
-        return new PubsubTopic(Type.FAKE, "", path);
-      }
-
-      String projectName, topicName;
-
-      Matcher v1beta1Match = V1BETA1_TOPIC_REGEXP.matcher(path);
-      if (v1beta1Match.matches()) {
-        LOG.warn("Saw topic in v1beta1 format.  Topics should be in the format "
-            + "projects/<project_id>/topics/<topic_name>");
-        projectName = v1beta1Match.group(1);
-        topicName = v1beta1Match.group(2);
-      } else {
-        Matcher match = TOPIC_REGEXP.matcher(path);
-        if (!match.matches()) {
-          throw new IllegalArgumentException(
-              "Pubsub topic is not in projects/<project_id>/topics/<topic_name> format: " + path);
-        }
-        projectName = match.group(1);
-        topicName = match.group(2);
-      }
-
-      validateProjectName(projectName);
-      validatePubsubName(topicName);
-      return new PubsubTopic(Type.NORMAL, projectName, topicName);
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * v1beta1 API.
-     *
-     * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta1Path() {
-      if (type == Type.NORMAL) {
-        return "/topics/" + project + "/" + topic;
-      } else {
-        return topic;
-      }
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * v1beta2 API.
-     *
-     * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated.
-     */
-    @Deprecated
-    public String asV1Beta2Path() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/topics/" + topic;
-      } else {
-        return topic;
-      }
-    }
-
-    /**
-     * Returns the string representation of this topic as a path used in the Cloud Pub/Sub
-     * API.
-     */
-    public String asPath() {
-      if (type == Type.NORMAL) {
-        return "projects/" + project + "/topics/" + topic;
-      } else {
-        return topic;
-      }
-    }
-  }
-
-  /**
-   * A {@link PTransform} that continuously reads from a Cloud Pub/Sub stream and
-   * returns a {@link PCollection} of {@link String Strings} containing the items from
-   * the stream.
-   *
-   * <p>When running with a {@link PipelineRunner} that only supports bounded
-   * {@link PCollection PCollections} (such as {@link DirectPipelineRunner}),
-   * only a bounded portion of the input Pub/Sub stream can be processed. As such, either
-   * {@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
-     * with {@link #subscription(String)}.
-     *
-     * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format
-     * of the {@code topic} string.
-     *
-     * <p>Dataflow will start reading data published on this topic from the time the pipeline is
-     * started. Any data published on the topic before the pipeline is started will not be read by
-     * Dataflow.
-     */
-    public static Bound<String> topic(String topic) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic);
-    }
-
-    /**
-     * Creates and returns a transform for reading from a specific Cloud Pub/Sub subscription.
-     * Mutually exclusive with {@link #topic(String)}.
-     *
-     * <p>See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format
-     * of the {@code subscription} string.
-     */
-    public static Bound<String> subscription(String subscription) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).subscription(subscription);
-    }
-
-    /**
-     * Creates and returns a transform reading from Cloud Pub/Sub where record timestamps are
-     * expected to be provided as Pub/Sub message attributes. The {@code timestampLabel}
-     * parameter specifies the name of the attribute that contains the timestamp.
-     *
-     * <p>The timestamp value is expected to be represented in the attribute as either:
-     *
-     * <ul>
-     * <li>a numerical value representing the number of milliseconds since the Unix epoch. For
-     * example, if using the Joda time classes, {@link Instant#getMillis()} returns the correct
-     * value for this attribute.
-     * <li>a String in RFC 3339 format. For example, {@code 2015-10-29T23:41:41.123Z}. The
-     * sub-second component of the timestamp is optional, and digits beyond the first three
-     * (i.e., time units smaller than milliseconds) will be ignored.
-     * </ul>
-     *
-     * <p>If {@code timestampLabel} is not provided, the system will generate record timestamps
-     * the first time it sees each record. All windowing will be done relative to these timestamps.
-     *
-     * <p>By default, windows are emitted based on an estimate of when this source is likely
-     * done producing data for a given timestamp (referred to as the Watermark; see
-     * {@link AfterWatermark} for more details). Any late data will be handled by the trigger
-     * specified with the windowing strategy &ndash; by default it will be output immediately.
-     *
-     * <p>Note that the system can guarantee that no late data will ever be seen when it assigns
-     * timestamps by arrival time (i.e. {@code timestampLabel} is not provided).
-     *
-     * @see <a href="https://www.ietf.org/rfc/rfc3339.txt">RFC 3339</a>
-     */
-    public static Bound<String> timestampLabel(String timestampLabel) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).timestampLabel(timestampLabel);
-    }
-
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub where unique record
-     * identifiers are expected to be provided as Pub/Sub message attributes. The {@code idLabel}
-     * parameter specifies the attribute name. The value of the attribute can be any string
-     * that uniquely identifies this record.
-     *
-     * <p>If {@code idLabel} is not provided, Dataflow cannot guarantee that no duplicate data will
-     * be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be
-     * strictly best effort.
-     */
-    public static Bound<String> idLabel(String idLabel) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel);
-    }
-
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub that uses the given
-     * {@link Coder} to decode Pub/Sub messages into a value of type {@code T}.
-     *
-     * <p>By default, uses {@link StringUtf8Coder}, which just
-     * returns the text lines as Java strings.
-     *
-     * @param <T> the type of the decoded elements, and the elements
-     * of the resulting PCollection.
-     */
-    public static <T> Bound<T> withCoder(Coder<T> coder) {
-      return new Bound<>(coder);
-    }
-
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub with a maximum number of
-     * records that will be read. The transform produces a <i>bounded</i> {@link PCollection}.
-     *
-     * <p>Either this option or {@link #maxReadTime(Duration)} must be set in order to create a
-     * bounded source.
-     */
-    public static Bound<String> maxNumRecords(int maxNumRecords) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).maxNumRecords(maxNumRecords);
-    }
-
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub with a maximum number of
-     * duration during which records will be read.  The transform produces a <i>bounded</i>
-     * {@link PCollection}.
-     *
-     * <p>Either this option or {@link #maxNumRecords(int)} must be set in order to create a bounded
-     * source.
-     */
-    public static Bound<String> maxReadTime(Duration maxReadTime) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).maxReadTime(maxReadTime);
-    }
-
-    /**
-     * A {@link PTransform} that reads from a Cloud Pub/Sub source and returns
-     * a unbounded {@link PCollection} containing the items from the stream.
-     */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
-      /** The Cloud Pub/Sub topic to read from. */
-      @Nullable private final PubsubTopic topic;
-
-      /** The Cloud Pub/Sub subscription to read from. */
-      @Nullable private final PubsubSubscription subscription;
-
-      /** The name of the message attribute to read timestamps from. */
-      @Nullable private final String timestampLabel;
-
-      /** The name of the message attribute to read unique message IDs from. */
-      @Nullable private final String idLabel;
-
-      /** The coder used to decode each record. */
-      @Nullable private final Coder<T> coder;
-
-      /** Stop after reading this many records. */
-      private final int maxNumRecords;
-
-      /** Stop after reading for this much time. */
-      @Nullable private final Duration maxReadTime;
-
-      private Bound(Coder<T> coder) {
-        this(null, null, null, null, coder, null, 0, null);
-      }
-
-      private Bound(String name, PubsubSubscription subscription, PubsubTopic topic,
-          String timestampLabel, Coder<T> coder, String idLabel, int maxNumRecords,
-          Duration maxReadTime) {
-        super(name);
-        this.subscription = subscription;
-        this.topic = topic;
-        this.timestampLabel = timestampLabel;
-        this.coder = coder;
-        this.idLabel = idLabel;
-        this.maxNumRecords = maxNumRecords;
-        this.maxReadTime = maxReadTime;
-      }
-
-      /**
-       * 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.
-       *
-       * <p>See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format
-       * of the {@code subscription} string.
-       *
-       * <p>Multiple readers reading from the same subscription will each receive
-       * some arbitrary portion of the data.  Most likely, separate readers should
-       * use their own subscriptions.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> subscription(String subscription) {
-        return new Bound<>(name, PubsubSubscription.fromPath(subscription), topic, timestampLabel,
-            coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but that reads from the specified topic.
-       *
-       * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the
-       * format of the {@code topic} string.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> topic(String topic) {
-        return new Bound<>(name, subscription, PubsubTopic.fromPath(topic), timestampLabel, coder,
-            idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but that reads message timestamps
-       * from the given message attribute. See {@link PubsubIO.Read#timestampLabel(String)} for
-       * more details on the format of the timestamp attribute.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> timestampLabel(String timestampLabel) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but that reads unique message IDs
-       * from the given message attribute. See {@link PubsubIO.Read#idLabel(String)} for more
-       * details on the format of the ID attribute.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> idLabel(String idLabel) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but that uses the given
-       * {@link Coder} to decode each record into a value of type {@code X}.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param <X> the type of the decoded elements, and the
-       * elements of the resulting PCollection.
-       */
-      public <X> Bound<X> withCoder(Coder<X> coder) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but will only read up to the specified
-       * maximum number of records from Cloud Pub/Sub. The transform produces a <i>bounded</i>
-       * {@link PCollection}. See {@link PubsubIO.Read#maxNumRecords(int)} for more details.
-       */
-      public Bound<T> maxNumRecords(int maxNumRecords) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
-       * Returns a transform that's like this one but will only read during the specified
-       * duration from Cloud Pub/Sub. The transform produces a <i>bounded</i> {@link PCollection}.
-       * See {@link PubsubIO.Read#maxReadTime(Duration)} for more details.
-       */
-      public Bound<T> maxReadTime(Duration maxReadTime) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      @Override
-      public PCollection<T> apply(PInput input) {
-        if (topic == null && subscription == null) {
-          throw new IllegalStateException("need to set either the topic or the subscription for "
-              + "a PubsubIO.Read transform");
-        }
-        if (topic != null && subscription != null) {
-          throw new IllegalStateException("Can't set both the topic and the subscription for a "
-              + "PubsubIO.Read transform");
-        }
-
-        boolean boundedOutput = getMaxNumRecords() > 0 || getMaxReadTime() != null;
-
-        if (boundedOutput) {
-          return input.getPipeline().begin()
-              .apply(Create.of((Void) null)).setCoder(VoidCoder.of())
-              .apply(ParDo.of(new PubsubReader())).setCoder(coder);
-        } else {
-          return PCollection.<T>createPrimitiveOutputInternal(
-                  input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED)
-              .setCoder(coder);
-        }
-      }
-
-      @Override
-      protected Coder<T> getDefaultOutputCoder() {
-        return coder;
-      }
-
-      public PubsubTopic getTopic() {
-        return topic;
-      }
-
-      public PubsubSubscription getSubscription() {
-        return subscription;
-      }
-
-      public String getTimestampLabel() {
-        return timestampLabel;
-      }
-
-      public Coder<T> getCoder() {
-        return coder;
-      }
-
-      public String getIdLabel() {
-        return idLabel;
-      }
-
-      public int getMaxNumRecords() {
-        return maxNumRecords;
-      }
-
-      public Duration getMaxReadTime() {
-        return maxReadTime;
-      }
-
-      private class PubsubReader extends DoFn<Void, T> {
-        private static final int DEFAULT_PULL_SIZE = 100;
-
-        @Override
-        public void processElement(ProcessContext c) throws IOException {
-          Pubsub pubsubClient =
-              Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class))
-                  .build();
-
-          String subscription;
-          if (getSubscription() == null) {
-            String topic = getTopic().asPath();
-            String[] split = topic.split("/");
-            subscription =
-                "projects/" + split[1] + "/subscriptions/" + split[3] + "_dataflow_"
-                + new Random().nextLong();
-            Subscription subInfo = new Subscription().setAckDeadlineSeconds(60).setTopic(topic);
-            try {
-              pubsubClient.projects().subscriptions().create(subscription, subInfo).execute();
-            } catch (Exception e) {
-              throw new RuntimeException("Failed to create subscription: ", e);
-            }
-          } else {
-            subscription = getSubscription().asPath();
-          }
-
-          Instant endTime = (getMaxReadTime() == null)
-              ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime());
-
-          List<PubsubMessage> messages = new ArrayList<>();
-
-          Throwable finallyBlockException = null;
-          try {
-            while ((getMaxNumRecords() == 0 || messages.size() < getMaxNumRecords())
-                && Instant.now().isBefore(endTime)) {
-              PullRequest pullRequest = new PullRequest().setReturnImmediately(false);
-              if (getMaxNumRecords() > 0) {
-                pullRequest.setMaxMessages(getMaxNumRecords() - messages.size());
-              } else {
-                pullRequest.setMaxMessages(DEFAULT_PULL_SIZE);
-              }
-
-              PullResponse pullResponse =
-                  pubsubClient.projects().subscriptions().pull(subscription, pullRequest).execute();
-              List<String> ackIds = new ArrayList<>();
-              if (pullResponse.getReceivedMessages() != null) {
-                for (ReceivedMessage received : pullResponse.getReceivedMessages()) {
-                  messages.add(received.getMessage());
-                  ackIds.add(received.getAckId());
-                }
-              }
-
-              if (ackIds.size() != 0) {
-                AcknowledgeRequest ackRequest = new AcknowledgeRequest().setAckIds(ackIds);
-                pubsubClient.projects()
-                    .subscriptions()
-                    .acknowledge(subscription, ackRequest)
-                    .execute();
-              }
-            }
-          } catch (IOException e) {
-            throw new RuntimeException("Unexpected exception while reading from Pubsub: ", e);
-          } finally {
-            if (getTopic() != null) {
-              try {
-                pubsubClient.projects().subscriptions().delete(subscription).execute();
-              } catch (IOException e) {
-                finallyBlockException = new RuntimeException("Failed to delete subscription: ", e);
-                LOG.error("Failed to delete subscription: ", e);
-              }
-            }
-          }
-          if (finallyBlockException != null) {
-            Throwables.propagate(finallyBlockException);
-          }
-
-          for (PubsubMessage message : messages) {
-            c.outputWithTimestamp(
-                CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()),
-                assignMessageTimestamp(message, getTimestampLabel(), Clock.SYSTEM));
-          }
-        }
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Read() {}
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Disallow construction of utility class. */
-  private PubsubIO() {}
-
-  /**
-   * A {@link PTransform} that continuously writes a
-   * {@link PCollection} of {@link String Strings} to a Cloud Pub/Sub stream.
-   */
-  // 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
-     * {@code topic} string.
-     */
-    public static Bound<String> topic(String topic) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic);
-    }
-
-    /**
-     * Creates a transform that writes to Pub/Sub, adds each record's timestamp to the published
-     * messages in an attribute with the specified name. The value of the attribute will be a number
-     * representing the number of milliseconds since the Unix epoch. For example, if using the Joda
-     * time classes, {@link Instant#Instant(long)} can be used to parse this value.
-     *
-     * <p>If the output from this sink is being read by another Dataflow source, then
-     * {@link PubsubIO.Read#timestampLabel(String)} can be used to ensure the other source reads
-     * these timestamps from the appropriate attribute.
-     */
-    public static Bound<String> timestampLabel(String timestampLabel) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).timestampLabel(timestampLabel);
-    }
-
-    /**
-     * Creates a transform that writes to Pub/Sub, adding each record's unique identifier to the
-     * published messages in an attribute with the specified name. The value of the attribute is an
-     * opaque string.
-     *
-     * <p>If the the output from this sink is being read by another Dataflow source, then
-     * {@link PubsubIO.Read#idLabel(String)} can be used to ensure that* the other source reads
-     * these unique identifiers from the appropriate attribute.
-     */
-    public static Bound<String> idLabel(String idLabel) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel);
-    }
-
-    /**
-     * Creates a transform that  uses the given {@link Coder} to encode each of the
-     * elements of the input collection into an output message.
-     *
-     * <p>By default, uses {@link StringUtf8Coder}, which writes input Java strings directly as
-     * records.
-     *
-     * @param <T> the type of the elements of the input PCollection
-     */
-    public static <T> Bound<T> withCoder(Coder<T> coder) {
-      return new Bound<>(coder);
-    }
-
-    /**
-     * A {@link PTransform} that writes an unbounded {@link PCollection} of {@link String Strings}
-     * to a Cloud Pub/Sub stream.
-     */
-    public static class Bound<T> extends PTransform<PCollection<T>, PDone> {
-      /** The Cloud Pub/Sub topic to publish to. */
-      @Nullable private final PubsubTopic topic;
-      /** The name of the message attribute to publish message timestamps in. */
-      @Nullable private final String timestampLabel;
-      /** The name of the message attribute to publish unique message IDs in. */
-      @Nullable private final String idLabel;
-      private final Coder<T> coder;
-
-      private Bound(Coder<T> coder) {
-        this(null, null, null, null, coder);
-      }
-
-      private Bound(
-          String name, PubsubTopic topic, String timestampLabel, String idLabel, Coder<T> coder) {
-        super(name);
-        this.topic = topic;
-        this.timestampLabel = timestampLabel;
-        this.idLabel = idLabel;
-        this.coder = coder;
-      }
-
-      /**
-       * 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.
-       *
-       * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the
-       * {@code topic} string.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> topic(String topic) {
-        return new Bound<>(name, PubsubTopic.fromPath(topic), timestampLabel, idLabel, coder);
-      }
-
-      /**
-       * Returns a new transform that's like this one but that publishes record timestamps
-       * to a message attribute with the specified name. See
-       * {@link PubsubIO.Write#timestampLabel(String)} for more details.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> timestampLabel(String timestampLabel) {
-        return new Bound<>(name, topic, timestampLabel, idLabel, coder);
-      }
-
-      /**
-       * Returns a new transform that's like this one but that publishes unique record IDs
-       * to a message attribute with the specified name. See {@link PubsubIO.Write#idLabel(String)}
-       * for more details.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> idLabel(String idLabel) {
-        return new Bound<>(name, topic, timestampLabel, idLabel, coder);
-      }
-
-      /**
-       * Returns a new transform that's like this one
-       * but that uses the given {@link Coder} to encode each of
-       * the elements of the input {@link PCollection} into an
-       * output record.
-       *
-       * <p>Does not modify this object.
-       *
-       * @param <X> the type of the elements of the input {@link PCollection}
-       */
-      public <X> Bound<X> withCoder(Coder<X> coder) {
-        return new Bound<>(name, topic, timestampLabel, idLabel, coder);
-      }
-
-      @Override
-      public PDone apply(PCollection<T> input) {
-        if (topic == null) {
-          throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform");
-        }
-        input.apply(ParDo.of(new PubsubWriter()));
-        return PDone.in(input.getPipeline());
-      }
-
-      @Override
-      protected Coder<Void> getDefaultOutputCoder() {
-        return VoidCoder.of();
-      }
-
-      public PubsubTopic getTopic() {
-        return topic;
-      }
-
-      public String getTimestampLabel() {
-        return timestampLabel;
-      }
-
-      public String getIdLabel() {
-        return idLabel;
-      }
-
-      public Coder<T> getCoder() {
-        return coder;
-      }
-
-      private class PubsubWriter extends DoFn<T, Void> {
-        private static final int MAX_PUBLISH_BATCH_SIZE = 100;
-        private transient List<PubsubMessage> output;
-        private transient Pubsub pubsubClient;
-
-        @Override
-        public void startBundle(Context c) {
-          this.output = new ArrayList<>();
-          this.pubsubClient =
-              Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class))
-                  .build();
-        }
-
-        @Override
-        public void processElement(ProcessContext c) throws IOException {
-          PubsubMessage message =
-              new PubsubMessage().encodeData(CoderUtils.encodeToByteArray(getCoder(), c.element()));
-          if (getTimestampLabel() != null) {
-            Map<String, String> attributes = message.getAttributes();
-            if (attributes == null) {
-              attributes = new HashMap<>();
-              message.setAttributes(attributes);
-            }
-            attributes.put(getTimestampLabel(), String.valueOf(c.timestamp().getMillis()));
-          }
-          output.add(message);
-
-          if (output.size() >= MAX_PUBLISH_BATCH_SIZE) {
-            publish();
-          }
-        }
-
-        @Override
-        public void finishBundle(Context c) throws IOException {
-          if (!output.isEmpty()) {
-            publish();
-          }
-        }
-
-        private void publish() throws IOException {
-          PublishRequest publishRequest = new PublishRequest().setMessages(output);
-          pubsubClient.projects().topics()
-              .publish(getTopic().asPath(), publishRequest)
-              .execute();
-          output.clear();
-        }
-      }
-    }
-
-    /** Disallow construction of utility class. */
-    private Write() {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java
deleted file mode 100644
index fb103ee..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java
+++ /dev/null
@@ -1,254 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PInput;
-
-import org.joda.time.Duration;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link PTransform} for reading from a {@link Source}.
- *
- * <p>Usage example:
- * <pre>
- * Pipeline p = Pipeline.create();
- * p.apply(Read.from(new MySource().withFoo("foo").withBar("bar"))
- *             .named("foobar"));
- * </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
-   * {@code BoundedSource}.
-   */
-  public static <T> Bounded<T> from(BoundedSource<T> source) {
-    return new Bounded<>(null, source);
-  }
-
-  /**
-   * Returns a new {@code Read.Unbounded} {@code PTransform} reading from the given
-   * {@code UnboundedSource}.
-   */
-  public static <T> Unbounded<T> from(UnboundedSource<T, ?> source) {
-    return new Unbounded<>(null, source);
-  }
-
-  /**
-   * Helper class for building {@code Read} transforms.
-   */
-  public static class Builder {
-    private final String name;
-
-    private Builder(String name) {
-      this.name = name;
-    }
-
-    /**
-     * Returns a new {@code Read.Bounded} {@code PTransform} reading from the given
-     * {@code BoundedSource}.
-     */
-    public <T> Bounded<T> from(BoundedSource<T> source) {
-      return new Bounded<>(name, source);
-    }
-
-    /**
-     * Returns a new {@code Read.Unbounded} {@code PTransform} reading from the given
-     * {@code UnboundedSource}.
-     */
-    public <T> Unbounded<T> from(UnboundedSource<T, ?> source) {
-      return new Unbounded<>(name, source);
-    }
-  }
-
-  /**
-   * {@link PTransform} that reads from a {@link BoundedSource}.
-   */
-  public static class Bounded<T> extends PTransform<PInput, PCollection<T>> {
-    private final BoundedSource<T> source;
-
-    private Bounded(@Nullable String name, BoundedSource<T> source) {
-      super(name);
-      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();
-    }
-
-    @Override
-    public final PCollection<T> apply(PInput input) {
-      source.validate();
-
-      return PCollection.<T>createPrimitiveOutputInternal(input.getPipeline(),
-          WindowingStrategy.globalDefault(), IsBounded.BOUNDED)
-          .setCoder(getDefaultOutputCoder());
-    }
-
-    /**
-     * Returns the {@code BoundedSource} used to create this {@code Read} {@code PTransform}.
-     */
-    public BoundedSource<T> getSource() {
-      return source;
-    }
-
-    @Override
-    public String getKindString() {
-      return "Read(" + approximateSimpleName(source.getClass()) + ")";
-    }
-
-    static {
-      registerDefaultTransformEvaluator();
-    }
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private static void registerDefaultTransformEvaluator() {
-      DirectPipelineRunner.registerDefaultTransformEvaluator(
-          Bounded.class,
-          new DirectPipelineRunner.TransformEvaluator<Bounded>() {
-            @Override
-            public void evaluate(
-                Bounded transform, DirectPipelineRunner.EvaluationContext context) {
-              evaluateReadHelper(transform, context);
-            }
-
-            private <T> void evaluateReadHelper(
-                Read.Bounded<T> transform, DirectPipelineRunner.EvaluationContext context) {
-              try {
-                List<DirectPipelineRunner.ValueWithMetadata<T>> output = new ArrayList<>();
-                BoundedSource<T> source = transform.getSource();
-                try (BoundedSource.BoundedReader<T> reader =
-                    source.createReader(context.getPipelineOptions())) {
-                  for (boolean available = reader.start();
-                      available;
-                      available = reader.advance()) {
-                    output.add(
-                        DirectPipelineRunner.ValueWithMetadata.of(
-                            WindowedValue.timestampedValueInGlobalWindow(
-                                reader.getCurrent(), reader.getCurrentTimestamp())));
-                  }
-                }
-                context.setPCollectionValuesWithMetadata(context.getOutput(transform), output);
-              } catch (Exception e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    }
-  }
-
-  /**
-   * {@link PTransform} that reads from a {@link UnboundedSource}.
-   */
-  public static class Unbounded<T> extends PTransform<PInput, PCollection<T>> {
-    private final UnboundedSource<T, ?> source;
-
-    private Unbounded(@Nullable String name, UnboundedSource<T, ?> source) {
-      super(name);
-      this.source = SerializableUtils.ensureSerializable(source);
-    }
-
-    /**
-     * 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.
-     *
-     * <p>This may take a long time to execute if the splits of this source are slow to read
-     * records.
-     */
-    public BoundedReadFromUnboundedSource<T> withMaxNumRecords(long maxNumRecords) {
-      return new BoundedReadFromUnboundedSource<T>(source, maxNumRecords, null);
-    }
-
-    /**
-     * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount
-     * of data from the given {@link UnboundedSource}.  The bound is specified as an amount
-     * of time to read for.  Each split of the source will read for this much time.
-     */
-    public BoundedReadFromUnboundedSource<T> withMaxReadTime(Duration maxReadTime) {
-      return new BoundedReadFromUnboundedSource<T>(source, Long.MAX_VALUE, maxReadTime);
-    }
-
-    @Override
-    protected Coder<T> getDefaultOutputCoder() {
-      return source.getDefaultOutputCoder();
-    }
-
-    @Override
-    public final PCollection<T> apply(PInput input) {
-      source.validate();
-
-      return PCollection.<T>createPrimitiveOutputInternal(
-          input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED);
-    }
-
-    /**
-     * Returns the {@code UnboundedSource} used to create this {@code Read} {@code PTransform}.
-     */
-    public UnboundedSource<T, ?> getSource() {
-      return source;
-    }
-
-    @Override
-    public String getKindString() {
-      return "Read(" + approximateSimpleName(source.getClass()) + ")";
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java
deleted file mode 100644
index cc233f1..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.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 com.google.cloud.dataflow.sdk.io;
-
-/**
- * Standard shard naming templates.
- *
- * <p>Shard naming templates are strings that may contain placeholders for
- * the shard number and shard count.  When constructing a filename for a
- * particular shard number, the upper-case letters 'S' and 'N' are replaced
- * with the 0-padded shard number and shard count respectively.
- *
- * <p>Left-padding of the numbers enables lexicographical sorting of the
- * resulting filenames.  If the shard number or count are too large for the
- * space provided in the template, then the result may no longer sort
- * lexicographically.  For example, a shard template of "S-of-N", for 200
- * shards, will result in outputs named "0-of-200", ... '10-of-200',
- * '100-of-200", etc.
- *
- * <p>Shard numbers start with 0, so the last shard number is the shard count
- * minus one.  For example, the template "-SSSSS-of-NNNNN" will be
- * instantiated as "-00000-of-01000" for the first shard (shard 0) of a
- * 1000-way sharded output.
- *
- * <p>A shard name template is typically provided along with a name prefix
- * and suffix, which allows constructing complex paths that have embedded
- * shard information.  For example, outputs in the form
- * "gs://bucket/path-01-of-99.txt" could be constructed by providing the
- * individual components:
- *
- * <pre>{@code
- *   pipeline.apply(
- *       TextIO.Write.to("gs://bucket/path")
- *                   .withShardNameTemplate("-SS-of-NN")
- *                   .withSuffix(".txt"))
- * }</pre>
- *
- * <p>In the example above, you could make parts of the output configurable
- * by users without the user having to specify all components of the output
- * name.
- *
- * <p>If a shard name template does not contain any repeating 'S', then
- * the output shard count must be 1, as otherwise the same filename would be
- * generated for multiple shards.
- */
-public class ShardNameTemplate {
-  /**
-   * Shard name containing the index and max.
-   *
-   * <p>Eg: [prefix]-00000-of-00100[suffix] and
-   * [prefix]-00001-of-00100[suffix]
-   */
-  public static final String INDEX_OF_MAX = "-SSSSS-of-NNNNN";
-
-  /**
-   * Shard is a file within a directory.
-   *
-   * <p>Eg: [prefix]/part-00000[suffix] and [prefix]/part-00001[suffix]
-   */
-  public static final String DIRECTORY_CONTAINER = "/part-SSSSS";
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java
deleted file mode 100644
index 7845f47..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java
+++ /dev/null
@@ -1,255 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.io.Serializable;
-
-/**
- * A {@code Sink} represents a resource that can be written to using the {@link Write} transform.
- *
- * <p>A parallel write to a {@code Sink} consists of three phases:
- * <ol>
- * <li>A sequential <i>initialization</i> phase (e.g., creating a temporary output directory, etc.)
- * <li>A <i>parallel write</i> phase where workers write bundles of records
- * <li>A sequential <i>finalization</i> phase (e.g., committing the writes, merging output files,
- * etc.)
- * </ol>
- *
- * <p>The {@link Write} transform can be used in a Dataflow pipeline to perform this write.
- * Specifically, a Write transform can be applied to a {@link PCollection} {@code p} by:
- *
- * <p>{@code p.apply(Write.to(new MySink()));}
- *
- * <p>Implementing a {@link Sink} and the corresponding write operations requires extending three
- * abstract classes:
- *
- * <ul>
- * <li>{@link Sink}: an immutable logical description of the location/resource to write to.
- * Depending on the type of sink, it may contain fields such as the path to an output directory
- * on a filesystem, a database table name, etc. Implementors of {@link Sink} must
- * implement two methods: {@link Sink#validate} and {@link Sink#createWriteOperation}.
- * {@link Sink#validate Validate} is called by the Write transform at pipeline creation, and should
- * validate that the Sink can be written to. The createWriteOperation method is also called at
- * pipeline creation, and should return a WriteOperation object that defines how to write to the
- * Sink. Note that implementations of Sink must be serializable and Sinks must be immutable.
- *
- * <li>{@link WriteOperation}: The WriteOperation implements the <i>initialization</i> and
- * <i>finalization</i> phases of a write. Implementors of {@link WriteOperation} must implement
- * corresponding {@link WriteOperation#initialize} and {@link WriteOperation#finalize} methods. A
- * WriteOperation must also implement {@link WriteOperation#createWriter} that creates Writers,
- * {@link WriteOperation#getWriterResultCoder} that returns a {@link Coder} for the result of a
- * parallel write, and a {@link WriteOperation#getSink} that returns the Sink that the write
- * operation corresponds to. See below for more information about these methods and restrictions on
- * their implementation.
- *
- * <li>{@link Writer}: A Writer writes a bundle of records. Writer defines four methods:
- * {@link Writer#open}, which is called once at the start of writing a bundle; {@link Writer#write},
- * which writes a single record from the bundle; {@link Writer#close}, which is called once at the
- * end of writing a bundle; and {@link Writer#getWriteOperation}, which returns the write operation
- * that the writer belongs to.
- * </ul>
- *
- * <h2>WriteOperation</h2>
- * <p>{@link WriteOperation#initialize} and {@link WriteOperation#finalize} are conceptually called
- * once: at the beginning and end of a Write transform. However, implementors must ensure that these
- * methods are idempotent, as they may be called multiple times on different machines in the case of
- * failure/retry or for redundancy.
- *
- * <p>The finalize method of WriteOperation is passed an Iterable of a writer result type. This
- * writer result type should encode the result of a write and, in most cases, some encoding of the
- * unique bundle id.
- *
- * <p>All implementations of {@link WriteOperation} must be serializable.
- *
- * <p>WriteOperation may have mutable state. For instance, {@link WriteOperation#initialize} may
- * mutate the object state. These mutations will be visible in {@link WriteOperation#createWriter}
- * and {@link WriteOperation#finalize} because the object will be serialized after initialize and
- * deserialized before these calls. However, it is not serialized again after createWriter is
- * called, as createWriter will be called within workers to create Writers for the bundles that are
- * distributed to these workers. Therefore, newWriter should not mutate the WriteOperation state (as
- * these mutations will not be visible in finalize).
- *
- * <h2>Bundle Ids:</h2>
- * <p>In order to ensure fault-tolerance, a bundle may be executed multiple times (e.g., in the
- * event of failure/retry or for redundancy). However, exactly one of these executions will have its
- * result passed to the WriteOperation's finalize method. Each call to {@link Writer#open} is passed
- * a unique <i>bundle id</i> when it is called by the Write transform, so even redundant or retried
- * bundles will have a unique way of identifying their output.
- *
- * <p>The bundle id should be used to guarantee that a bundle's output is unique. This uniqueness
- * guarantee is important; if a bundle is to be output to a file, for example, the name of the file
- * must be unique to avoid conflicts with other Writers. The bundle id should be encoded in the
- * writer result returned by the Writer and subsequently used by the WriteOperation's finalize
- * method to identify the results of successful writes.
- *
- * <p>For example, consider the scenario where a Writer writes files containing serialized records
- * and the WriteOperation's finalization step is to merge or rename these output files. In this
- * case, a Writer may use its unique id to name its output file (to avoid conflicts) and return the
- * name of the file it wrote as its writer result. The WriteOperation will then receive an Iterable
- * of output file names that it can then merge or rename using some bundle naming scheme.
- *
- * <h2>Writer Results:</h2>
- * <p>{@link WriteOperation}s and {@link Writer}s must agree on a writer result type that will be
- * returned by a Writer after it writes a bundle. This type can be a client-defined object or an
- * existing type; {@link WriteOperation#getWriterResultCoder} should return a {@link Coder} for the
- * type.
- *
- * <p>A note about thread safety: Any use of static members or methods in Writer should be thread
- * safe, as different instances of Writer objects may be created in different threads on the same
- * worker.
- *
- * @param <T> the type that will be written to the Sink.
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public abstract class Sink<T> implements Serializable {
-  /**
-   * Ensures that the sink is valid and can be written to before the write operation begins. One
-   * should use {@link com.google.common.base.Preconditions} to implement this method.
-   */
-  public abstract void validate(PipelineOptions options);
-
-  /**
-   * Returns an instance of a {@link WriteOperation} that can write to this Sink.
-   */
-  public abstract WriteOperation<T, ?> createWriteOperation(PipelineOptions options);
-
-  /**
-   * A {@link WriteOperation} defines the process of a parallel write of objects to a Sink.
-   *
-   * <p>The {@code WriteOperation} defines how to perform initialization and finalization of a
-   * parallel write to a sink as well as how to create a {@link Sink.Writer} object that can write
-   * a bundle to the sink.
-   *
-   * <p>Since operations in Dataflow may be run multiple times for redundancy or fault-tolerance,
-   * the initialization and finalization defined by a WriteOperation <b>must be idempotent</b>.
-   *
-   * <p>{@code WriteOperation}s may be mutable; a {@code WriteOperation} is serialized after the
-   * call to {@code initialize} method and deserialized before calls to
-   * {@code createWriter} and {@code finalized}. However, it is not
-   * reserialized after {@code createWriter}, so {@code createWriter} should not mutate the
-   * state of the {@code WriteOperation}.
-   *
-   * <p>See {@link Sink} for more detailed documentation about the process of writing to a Sink.
-   *
-   * @param <T> The type of objects to write
-   * @param <WriteT> The result of a per-bundle write
-   */
-  public abstract static class WriteOperation<T, WriteT> implements Serializable {
-    /**
-     * Performs initialization before writing to the sink. Called before writing begins.
-     */
-    public abstract void initialize(PipelineOptions options) throws Exception;
-
-    /**
-     * Given an Iterable of results from bundle writes, performs finalization after writing and
-     * closes the sink. Called after all bundle writes are complete.
-     *
-     * <p>The results that are passed to finalize are those returned by bundles that completed
-     * successfully. Although bundles may have been run multiple times (for fault-tolerance), only
-     * one writer result will be passed to finalize for each bundle. An implementation of finalize
-     * should perform clean up of any failed and successfully retried bundles.  Note that these
-     * failed bundles will not have their writer result passed to finalize, so finalize should be
-     * capable of locating any temporary/partial output written by failed bundles.
-     *
-     * <p>A best practice is to make finalize atomic. If this is impossible given the semantics
-     * of the sink, finalize should be idempotent, as it may be called multiple times in the case of
-     * failure/retry or for redundancy.
-     *
-     * <p>Note that the iteration order of the writer results is not guaranteed to be consistent if
-     * finalize is called multiple times.
-     *
-     * @param writerResults an Iterable of results from successful bundle writes.
-     */
-    public abstract void finalize(Iterable<WriteT> writerResults, PipelineOptions options)
-        throws Exception;
-
-    /**
-     * Creates a new {@link Sink.Writer} to write a bundle of the input to the sink.
-     *
-     * <p>The bundle id that the writer will use to uniquely identify its output will be passed to
-     * {@link Writer#open}.
-     *
-     * <p>Must not mutate the state of the WriteOperation.
-     */
-    public abstract Writer<T, WriteT> createWriter(PipelineOptions options) throws Exception;
-
-    /**
-     * Returns the Sink that this write operation writes to.
-     */
-    public abstract Sink<T> getSink();
-
-    /**
-     * Returns a coder for the writer result type.
-     */
-    public Coder<WriteT> getWriterResultCoder() {
-      return null;
-    }
-  }
-
-  /**
-   * A Writer writes a bundle of elements from a PCollection to a sink. {@link Writer#open} is
-   * called before writing begins and {@link Writer#close} is called after all elements in the
-   * bundle have been written. {@link Writer#write} writes an element to the sink.
-   *
-   * <p>Note that any access to static members or methods of a Writer must be thread-safe, as
-   * multiple instances of a Writer may be instantiated in different threads on the same worker.
-   *
-   * <p>See {@link Sink} for more detailed documentation about the process of writing to a Sink.
-   *
-   * @param <T> The type of object to write
-   * @param <WriteT> The writer results type (e.g., the bundle's output filename, as String)
-   */
-  public abstract static class Writer<T, WriteT> {
-    /**
-     * Performs bundle initialization. For example, creates a temporary file for writing or
-     * initializes any state that will be used across calls to {@link Writer#write}.
-     *
-     * <p>The unique id that is given to open should be used to ensure that the writer's output does
-     * not interfere with the output of other Writers, as a bundle may be executed many times for
-     * fault tolerance. See {@link Sink} for more information about bundle ids.
-     */
-    public abstract void open(String uId) throws Exception;
-
-    /**
-     * Called for each value in the bundle.
-     */
-    public abstract void write(T value) throws Exception;
-
-    /**
-     * Finishes writing the bundle. Closes any resources used for writing the bundle.
-     *
-     * <p>Returns a writer result that will be used in the {@link Sink.WriteOperation}'s
-     * finalization. The result should contain some way to identify the output of this bundle (using
-     * the bundle id). {@link WriteOperation#finalize} will use the writer result to identify
-     * successful writes. See {@link Sink} for more information about bundle ids.
-     *
-     * @return the writer result
-     */
-    public abstract WriteT close() throws Exception;
-
-    /**
-     * Returns the write operation this writer belongs to.
-     */
-    public abstract WriteOperation<T, WriteT> getWriteOperation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java
deleted file mode 100644
index 86a8e64..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java
+++ /dev/null
@@ -1,194 +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 com.google.cloud.dataflow.sdk.io;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.NoSuchElementException;
-
-/**
- * Base class for defining input formats and creating a {@code Source} for reading the input.
- *
- * <p>This class is not intended to be subclassed directly. Instead, to define
- * a bounded source (a source which produces a finite amount of input), subclass
- * {@link BoundedSource}; to define an unbounded source, subclass {@link UnboundedSource}.
- *
- * <p>A {@code Source} passed to a {@code Read} transform must be
- * {@code Serializable}.  This allows the {@code Source} instance
- * created in this "main program" to be sent (in serialized form) to
- * remote worker machines and reconstituted for each batch of elements
- * of the input {@code PCollection} being processed or for each source splitting
- * operation. A {@code Source} can have instance variable state, and
- * non-transient instance variable state will be serialized in the main program
- * and then deserialized on remote worker machines.
- *
- * <p>{@code Source} classes MUST be effectively immutable. The only acceptable use of
- * mutable fields is to cache the results of expensive operations, and such fields MUST be
- * marked {@code transient}.
- *
- * <p>{@code Source} objects should override {@link Object#toString}, as it will be
- * used in important error and debugging messages.
- *
- * @param <T> Type of elements read by the source.
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public abstract class Source<T> implements Serializable {
-  /**
-   * Checks that this source is valid, before it can be used in a pipeline.
-   *
-   * <p>It is recommended to use {@link com.google.common.base.Preconditions} for implementing
-   * this method.
-   */
-  public abstract void validate();
-
-  /**
-   * Returns the default {@code Coder} to use for the data read from this source.
-   */
-  public abstract Coder<T> getDefaultOutputCoder();
-
-  /**
-   * The interface that readers of custom input sources must implement.
-   *
-   * <p>This interface is deliberately distinct from {@link java.util.Iterator} because
-   * the current model tends to be easier to program and more efficient in practice
-   * for iterating over sources such as files, databases etc. (rather than pure collections).
-   *
-   * <p>Reading data from the {@link Reader} must obey the following access pattern:
-   * <ul>
-   * <li> One call to {@link #start}
-   * <ul><li>If {@link #start} returned true, any number of calls to {@code getCurrent}*
-   *   methods</ul>
-   * <li> Repeatedly, a call to {@link #advance}. This may be called regardless
-   *   of what the previous {@link #start}/{@link #advance} returned.
-   * <ul><li>If {@link #advance} returned true, any number of calls to {@code getCurrent}*
-   *   methods</ul>
-   * </ul>
-   *
-   * <p>For example, if the reader is reading a fixed set of data:
-   * <pre>
-   *   try {
-   *     for (boolean available = reader.start(); available; available = reader.advance()) {
-   *       T item = reader.getCurrent();
-   *       Instant timestamp = reader.getCurrentTimestamp();
-   *       ...
-   *     }
-   *   } finally {
-   *     reader.close();
-   *   }
-   * </pre>
-   *
-   * <p>If the set of data being read is continually growing:
-   * <pre>
-   *   try {
-   *     boolean available = reader.start();
-   *     while (true) {
-   *       if (available) {
-   *         T item = reader.getCurrent();
-   *         Instant timestamp = reader.getCurrentTimestamp();
-   *         ...
-   *         resetExponentialBackoff();
-   *       } else {
-   *         exponentialBackoff();
-   *       }
-   *       available = reader.advance();
-   *     }
-   *   } finally {
-   *     reader.close();
-   *   }
-   * </pre>
-   *
-   * <p>Note: this interface is a work-in-progress and may change.
-   *
-   * <p>All {@code Reader} functions except {@link #getCurrentSource} do not need to be thread-safe;
-   * they may only be accessed by a single thread at once. However, {@link #getCurrentSource} needs
-   * to be thread-safe, and other functions should assume that its returned value can change
-   * asynchronously.
-   */
-  public abstract static class Reader<T> implements AutoCloseable {
-    /**
-     * Initializes the reader and advances the reader to the first record.
-     *
-     * <p>This method should be called exactly once. The invocation should occur prior to calling
-     * {@link #advance} or {@link #getCurrent}. This method may perform expensive operations that
-     * are needed to initialize the reader.
-     *
-     * @return {@code true} if a record was read, {@code false} if there is no more input available.
-     */
-    public abstract boolean start() throws IOException;
-
-    /**
-     * Advances the reader to the next valid record.
-     *
-     * <p>It is an error to call this without having called {@link #start} first.
-     *
-     * @return {@code true} if a record was read, {@code false} if there is no more input available.
-     */
-    public abstract boolean advance() throws IOException;
-
-    /**
-     * Returns the value of the data item that was read by the last {@link #start} or
-     * {@link #advance} call. The returned value must be effectively immutable and remain valid
-     * indefinitely.
-     *
-     * <p>Multiple calls to this method without an intervening call to {@link #advance} should
-     * return the same result.
-     *
-     * @throws java.util.NoSuchElementException if {@link #start} was never called, or if
-     *         the last {@link #start} or {@link #advance} returned {@code false}.
-     */
-    public abstract T getCurrent() throws NoSuchElementException;
-
-    /**
-     * Returns the timestamp associated with the current data item.
-     *
-     * <p>If the source does not support timestamps, this should return
-     * {@code BoundedWindow.TIMESTAMP_MIN_VALUE}.
-     *
-     * <p>Multiple calls to this method without an intervening call to {@link #advance} should
-     * return the same result.
-     *
-     * @throws NoSuchElementException if the reader is at the beginning of the input and
-     *         {@link #start} or {@link #advance} wasn't called, or if the last {@link #start} or
-     *         {@link #advance} returned {@code false}.
-     */
-    public abstract Instant getCurrentTimestamp() throws NoSuchElementException;
-
-    /**
-     * Closes the reader. The reader cannot be used after this method is called.
-     */
-    @Override
-    public abstract void close() throws IOException;
-
-    /**
-     * Returns a {@code Source} describing the same input that this {@code Reader} currently reads
-     * (including items already read).
-     *
-     * <p>Usually, an implementation will simply return the immutable {@link Source} object from
-     * which the current {@link Reader} was constructed, or delegate to the base class.
-     * However, when using or implementing this method on a {@link BoundedSource.BoundedReader},
-     * special considerations apply, see documentation for
-     * {@link BoundedSource.BoundedReader#getCurrentSource}.
-     */
-    public abstract Source<T> getCurrentSource();
-  }
-}


[09/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
deleted file mode 100644
index 9e7c16e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
+++ /dev/null
@@ -1,442 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.Registration;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
-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.ImmutableMap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.MutableClassToInstanceMap;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.JavaType;
-import com.fasterxml.jackson.databind.JsonDeserializer;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.JsonSerializer;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-import java.beans.PropertyDescriptor;
-import java.io.IOException;
-import java.lang.annotation.Annotation;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.lang.reflect.Type;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import javax.annotation.concurrent.ThreadSafe;
-
-/**
- * Represents and {@link InvocationHandler} for a {@link Proxy}. The invocation handler uses bean
- * introspection of the proxy class to store and retrieve values based off of the property name.
- *
- * <p>Unset properties use the {@code @Default} metadata on the getter to return values. If there
- * is no {@code @Default} annotation on the getter, then a <a
- * href="https://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html">default</a> as
- * per the Java Language Specification for the expected return type is returned.
- *
- * <p>In addition to the getter/setter pairs, this proxy invocation handler supports
- * {@link Object#equals(Object)}, {@link Object#hashCode()}, {@link Object#toString()} and
- * {@link PipelineOptions#as(Class)}.
- */
-@ThreadSafe
-class ProxyInvocationHandler implements InvocationHandler {
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-  /**
-   * No two instances of this class are considered equivalent hence we generate a random hash code
-   * between 0 and {@link Integer#MAX_VALUE}.
-   */
-  private final int hashCode = (int) (Math.random() * Integer.MAX_VALUE);
-  private final Set<Class<? extends PipelineOptions>> knownInterfaces;
-  private final ClassToInstanceMap<PipelineOptions> interfaceToProxyCache;
-  private final Map<String, Object> options;
-  private final Map<String, JsonNode> jsonOptions;
-  private final Map<String, String> gettersToPropertyNames;
-  private final Map<String, String> settersToPropertyNames;
-
-  ProxyInvocationHandler(Map<String, Object> options) {
-    this(options, Maps.<String, JsonNode>newHashMap());
-  }
-
-  private ProxyInvocationHandler(Map<String, Object> options, Map<String, JsonNode> jsonOptions) {
-    this.options = options;
-    this.jsonOptions = jsonOptions;
-    this.knownInterfaces = new HashSet<>(PipelineOptionsFactory.getRegisteredOptions());
-    gettersToPropertyNames = Maps.newHashMap();
-    settersToPropertyNames = Maps.newHashMap();
-    interfaceToProxyCache = MutableClassToInstanceMap.create();
-  }
-
-  @Override
-  public Object invoke(Object proxy, Method method, Object[] args) {
-    if (args == null && "toString".equals(method.getName())) {
-      return toString();
-    } else if (args != null && args.length == 1 && "equals".equals(method.getName())) {
-      return equals(args[0]);
-    } else if (args == null && "hashCode".equals(method.getName())) {
-      return hashCode();
-    } else if (args != null && "as".equals(method.getName()) && args[0] instanceof Class) {
-      @SuppressWarnings("unchecked")
-      Class<? extends PipelineOptions> clazz = (Class<? extends PipelineOptions>) args[0];
-      return as(clazz);
-    } else if (args != null && "cloneAs".equals(method.getName()) && args[0] instanceof Class) {
-      @SuppressWarnings("unchecked")
-      Class<? extends PipelineOptions> clazz = (Class<? extends PipelineOptions>) args[0];
-      return cloneAs(proxy, clazz);
-    }
-    String methodName = method.getName();
-    synchronized (this) {
-      if (gettersToPropertyNames.keySet().contains(methodName)) {
-        String propertyName = gettersToPropertyNames.get(methodName);
-        if (!options.containsKey(propertyName)) {
-          // Lazy bind the default to the method.
-          Object value = jsonOptions.containsKey(propertyName)
-              ? getValueFromJson(propertyName, method)
-              : getDefault((PipelineOptions) proxy, method);
-          options.put(propertyName, value);
-        }
-        return options.get(propertyName);
-      } else if (settersToPropertyNames.containsKey(methodName)) {
-        options.put(settersToPropertyNames.get(methodName), args[0]);
-        return Void.TYPE;
-      }
-    }
-    throw new RuntimeException("Unknown method [" + method + "] invoked with args ["
-        + Arrays.toString(args) + "].");
-  }
-
-  /**
-   * Backing implementation for {@link PipelineOptions#as(Class)}.
-   *
-   * @param iface The interface that the returned object needs to implement.
-   * @return An object that implements the interface <T>.
-   */
-  synchronized <T extends PipelineOptions> T as(Class<T> iface) {
-    Preconditions.checkNotNull(iface);
-    Preconditions.checkArgument(iface.isInterface());
-    if (!interfaceToProxyCache.containsKey(iface)) {
-      Registration<T> registration =
-          PipelineOptionsFactory.validateWellFormed(iface, knownInterfaces);
-      List<PropertyDescriptor> propertyDescriptors = registration.getPropertyDescriptors();
-      Class<T> proxyClass = registration.getProxyClass();
-      gettersToPropertyNames.putAll(generateGettersToPropertyNames(propertyDescriptors));
-      settersToPropertyNames.putAll(generateSettersToPropertyNames(propertyDescriptors));
-      knownInterfaces.add(iface);
-      interfaceToProxyCache.putInstance(iface,
-          InstanceBuilder.ofType(proxyClass)
-              .fromClass(proxyClass)
-              .withArg(InvocationHandler.class, this)
-              .build());
-    }
-    return interfaceToProxyCache.getInstance(iface);
-  }
-
-  /**
-   * Backing implementation for {@link PipelineOptions#cloneAs(Class)}.
-   *
-   * @return A copy of the PipelineOptions.
-   */
-  synchronized <T extends PipelineOptions> T cloneAs(Object proxy, Class<T> iface) {
-    PipelineOptions clonedOptions;
-    try {
-      clonedOptions = MAPPER.readValue(MAPPER.writeValueAsBytes(proxy), PipelineOptions.class);
-    } catch (IOException e) {
-      throw new IllegalStateException("Failed to serialize the pipeline options to JSON.", e);
-    }
-    for (Class<? extends PipelineOptions> knownIface : knownInterfaces) {
-      clonedOptions.as(knownIface);
-    }
-    return clonedOptions.as(iface);
-  }
-
-  /**
-   * Returns true if the other object is a ProxyInvocationHandler or is a Proxy object and has the
-   * same ProxyInvocationHandler as this.
-   *
-   * @param obj The object to compare against this.
-   * @return true iff the other object is a ProxyInvocationHandler or is a Proxy object and has the
-   *         same ProxyInvocationHandler as this.
-   */
-  @Override
-  public boolean equals(Object obj) {
-    return obj != null && ((obj instanceof ProxyInvocationHandler && this == obj)
-        || (Proxy.isProxyClass(obj.getClass()) && this == Proxy.getInvocationHandler(obj)));
-  }
-
-  /**
-   * Each instance of this ProxyInvocationHandler is unique and has a random hash code.
-   *
-   * @return A hash code that was generated randomly.
-   */
-  @Override
-  public int hashCode() {
-    return hashCode;
-  }
-
-  /**
-   * This will output all the currently set values. This is a relatively costly function
-   * as it will call {@code toString()} on each object that has been set and format
-   * the results in a readable format.
-   *
-   * @return A pretty printed string representation of this.
-   */
-  @Override
-  public synchronized String toString() {
-    SortedMap<String, Object> sortedOptions = new TreeMap<>();
-    // Add the options that we received from deserialization
-    sortedOptions.putAll(jsonOptions);
-    // Override with any programmatically set options.
-    sortedOptions.putAll(options);
-
-    StringBuilder b = new StringBuilder();
-    b.append("Current Settings:\n");
-    for (Map.Entry<String, Object> entry : sortedOptions.entrySet()) {
-      b.append("  " + entry.getKey() + ": " + entry.getValue() + "\n");
-    }
-    return b.toString();
-  }
-
-  /**
-   * Uses a Jackson {@link ObjectMapper} to attempt type conversion.
-   *
-   * @param method The method whose return type you would like to return.
-   * @param propertyName The name of the property that is being returned.
-   * @return An object matching the return type of the method passed in.
-   */
-  private Object getValueFromJson(String propertyName, Method method) {
-    try {
-      JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType());
-      JsonNode jsonNode = jsonOptions.get(propertyName);
-      return MAPPER.readValue(jsonNode.toString(), type);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to parse representation", e);
-    }
-  }
-
-  /**
-   * Returns a default value for the method based upon {@code @Default} metadata on the getter
-   * to return values. If there is no {@code @Default} annotation on the getter, then a <a
-   * href="https://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html">default</a> as
-   * per the Java Language Specification for the expected return type is returned.
-   *
-   * @param proxy The proxy object for which we are attempting to get the default.
-   * @param method The getter method that was invoked.
-   * @return The default value from an {@link Default} annotation if present, otherwise a default
-   *         value as per the Java Language Specification.
-   */
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private Object getDefault(PipelineOptions proxy, Method method) {
-    for (Annotation annotation : method.getAnnotations()) {
-      if (annotation instanceof Default.Class) {
-        return ((Default.Class) annotation).value();
-      } else if (annotation instanceof Default.String) {
-        return ((Default.String) annotation).value();
-      } else if (annotation instanceof Default.Boolean) {
-        return ((Default.Boolean) annotation).value();
-      } else if (annotation instanceof Default.Character) {
-        return ((Default.Character) annotation).value();
-      } else if (annotation instanceof Default.Byte) {
-        return ((Default.Byte) annotation).value();
-      } else if (annotation instanceof Default.Short) {
-        return ((Default.Short) annotation).value();
-      } else if (annotation instanceof Default.Integer) {
-        return ((Default.Integer) annotation).value();
-      } else if (annotation instanceof Default.Long) {
-        return ((Default.Long) annotation).value();
-      } else if (annotation instanceof Default.Float) {
-        return ((Default.Float) annotation).value();
-      } else if (annotation instanceof Default.Double) {
-        return ((Default.Double) annotation).value();
-      } else if (annotation instanceof Default.Enum) {
-        return Enum.valueOf((Class<Enum>) method.getReturnType(),
-            ((Default.Enum) annotation).value());
-      } else if (annotation instanceof Default.InstanceFactory) {
-        return InstanceBuilder.ofType(((Default.InstanceFactory) annotation).value())
-            .build()
-            .create(proxy);
-      }
-    }
-
-    /*
-     * We need to make sure that we return something appropriate for the return type. Thus we return
-     * a default value as defined by the JLS.
-     */
-    return Defaults.defaultValue(method.getReturnType());
-  }
-
-  /**
-   * Returns a map from the getters method name to the name of the property based upon the passed in
-   * {@link PropertyDescriptor}s property descriptors.
-   *
-   * @param propertyDescriptors A list of {@link PropertyDescriptor}s to use when generating the
-   *        map.
-   * @return A map of getter method name to property name.
-   */
-  private static Map<String, String> generateGettersToPropertyNames(
-      List<PropertyDescriptor> propertyDescriptors) {
-    ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
-    for (PropertyDescriptor descriptor : propertyDescriptors) {
-      if (descriptor.getReadMethod() != null) {
-        builder.put(descriptor.getReadMethod().getName(), descriptor.getName());
-      }
-    }
-    return builder.build();
-  }
-
-  /**
-   * Returns a map from the setters method name to its matching getters method name based upon the
-   * passed in {@link PropertyDescriptor}s property descriptors.
-   *
-   * @param propertyDescriptors A list of {@link PropertyDescriptor}s to use when generating the
-   *        map.
-   * @return A map of setter method name to getter method name.
-   */
-  private static Map<String, String> generateSettersToPropertyNames(
-      List<PropertyDescriptor> propertyDescriptors) {
-    ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
-    for (PropertyDescriptor descriptor : propertyDescriptors) {
-      if (descriptor.getWriteMethod() != null) {
-        builder.put(descriptor.getWriteMethod().getName(), descriptor.getName());
-      }
-    }
-    return builder.build();
-  }
-
-  static class Serializer extends JsonSerializer<PipelineOptions> {
-    @Override
-    public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvider provider)
-        throws IOException, JsonProcessingException {
-      ProxyInvocationHandler handler = (ProxyInvocationHandler) Proxy.getInvocationHandler(value);
-      synchronized (handler) {
-        // We first filter out any properties that have been modified since
-        // the last serialization of this PipelineOptions and then verify that
-        // they are all serializable.
-        Map<String, Object> filteredOptions = Maps.newHashMap(handler.options);
-        removeIgnoredOptions(handler.knownInterfaces, filteredOptions);
-        ensureSerializable(handler.knownInterfaces, filteredOptions);
-
-        // Now we create the map of serializable options by taking the original
-        // set of serialized options (if any) and updating them with any properties
-        // instances that have been modified since the previous serialization.
-        Map<String, Object> serializableOptions =
-            Maps.<String, Object>newHashMap(handler.jsonOptions);
-        serializableOptions.putAll(filteredOptions);
-        jgen.writeStartObject();
-        jgen.writeFieldName("options");
-        jgen.writeObject(serializableOptions);
-        jgen.writeEndObject();
-      }
-    }
-
-    /**
-     * We remove all properties within the passed in options where there getter is annotated with
-     * {@link JsonIgnore @JsonIgnore} from the passed in options using the passed in interfaces.
-     */
-    private void removeIgnoredOptions(
-        Set<Class<? extends PipelineOptions>> interfaces, Map<String, Object> options) {
-      // Find all the method names that are annotated with JSON ignore.
-      Set<String> jsonIgnoreMethodNames = FluentIterable.from(
-          ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces))
-          .filter(JsonIgnorePredicate.INSTANCE).transform(new Function<Method, String>() {
-            @Override
-            public String apply(Method input) {
-              return input.getName();
-            }
-          }).toSet();
-
-      // Remove all options that have the same method name as the descriptor.
-      for (PropertyDescriptor descriptor
-          : PipelineOptionsFactory.getPropertyDescriptors(interfaces)) {
-        if (jsonIgnoreMethodNames.contains(descriptor.getReadMethod().getName())) {
-          options.remove(descriptor.getName());
-        }
-      }
-    }
-
-    /**
-     * We use an {@link ObjectMapper} to verify that the passed in options are serializable
-     * and deserializable.
-     */
-    private void ensureSerializable(Set<Class<? extends PipelineOptions>> interfaces,
-        Map<String, Object> options) throws IOException {
-      // Construct a map from property name to the return type of the getter.
-      Map<String, Type> propertyToReturnType = Maps.newHashMap();
-      for (PropertyDescriptor descriptor
-          : PipelineOptionsFactory.getPropertyDescriptors(interfaces)) {
-        if (descriptor.getReadMethod() != null) {
-          propertyToReturnType.put(descriptor.getName(),
-              descriptor.getReadMethod().getGenericReturnType());
-        }
-      }
-
-      // Attempt to serialize and deserialize each property.
-      for (Map.Entry<String, Object> entry : options.entrySet()) {
-        try {
-          String serializedValue = MAPPER.writeValueAsString(entry.getValue());
-          JavaType type = MAPPER.getTypeFactory()
-              .constructType(propertyToReturnType.get(entry.getKey()));
-          MAPPER.readValue(serializedValue, type);
-        } catch (Exception e) {
-          throw new IOException(String.format(
-              "Failed to serialize and deserialize property '%s' with value '%s'",
-              entry.getKey(), entry.getValue()), e);
-        }
-      }
-    }
-  }
-
-  static class Deserializer extends JsonDeserializer<PipelineOptions> {
-    @Override
-    public PipelineOptions deserialize(JsonParser jp, DeserializationContext ctxt)
-        throws IOException, JsonProcessingException {
-      ObjectNode objectNode = (ObjectNode) jp.readValueAsTree();
-      ObjectNode optionsNode = (ObjectNode) objectNode.get("options");
-
-      Map<String, JsonNode> fields = Maps.newHashMap();
-      for (Iterator<Map.Entry<String, JsonNode>> iterator = optionsNode.fields();
-          iterator.hasNext(); ) {
-        Map.Entry<String, JsonNode> field = iterator.next();
-        fields.put(field.getKey(), field.getValue());
-      }
-      PipelineOptions options =
-          new ProxyInvocationHandler(Maps.<String, Object>newHashMap(), fields)
-              .as(PipelineOptions.class);
-      return options;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PubsubOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PubsubOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PubsubOptions.java
deleted file mode 100644
index deb19e9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PubsubOptions.java
+++ /dev/null
@@ -1,36 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * Properties that can be set when using Pubsub with the Beam SDK.
- */
-@Description("Options that are used to configure BigQuery. See "
-    + "https://cloud.google.com/bigquery/what-is-bigquery for details on BigQuery.")
-public interface PubsubOptions extends ApplicationNameOptions, GcpOptions,
-    PipelineOptions, StreamingOptions {
-
-  /**
-   * Root URL for use with the Pubsub API.
-   */
-  @Description("Root URL for use with the Pubsub API")
-  @Default.String("https://pubsub.googleapis.com")
-  @Hidden
-  String getPubsubRootUrl();
-  void setPubsubRootUrl(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
deleted file mode 100644
index 67cb386..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
+++ /dev/null
@@ -1,31 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * Options used to configure streaming.
- */
-public interface StreamingOptions extends
-    ApplicationNameOptions, GcpOptions, PipelineOptions {
-  /**
-   * Set to true if running a streaming pipeline.
-   */
-  @Description("Set to true if running a streaming pipeline.")
-  boolean isStreaming();
-  void setStreaming(boolean value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java
deleted file mode 100644
index b7725a6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.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 com.google.cloud.dataflow.sdk.options;
-
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * {@link Validation} represents a set of annotations that can be used to annotate getter
- * properties on {@link PipelineOptions} with information representing the validation criteria to
- * be used when validating with the {@link PipelineOptionsValidator}.
- */
-public @interface Validation {
-  /**
-   * This criteria specifies that the value must be not null. Note that this annotation
-   * should only be applied to methods that return nullable objects.
-   */
-  @Target(value = ElementType.METHOD)
-  @Retention(RetentionPolicy.RUNTIME)
-  @Documented
-  public @interface Required {
-    /**
-     * The groups that the annotated attribute is a member of. A member can be in 0 or more groups.
-     * Members not in any groups are considered to be in a group consisting exclusively of
-     * themselves. At least one member of a group must be non-null if the options are to be valid.
-     */
-    String[] groups() default {};
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java
deleted file mode 100644
index 63a03f5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java
+++ /dev/null
@@ -1,26 +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.
- */
-/**
- * Defines {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} for
- * configuring pipeline execution.
- *
- * <p>{@link com.google.cloud.dataflow.sdk.options.PipelineOptions} encapsulates the various
- * parameters that describe how a pipeline should be run. {@code PipelineOptions} are created
- * using a {@link com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory}.
- */
-package com.google.cloud.dataflow.sdk.options;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/package-info.java
deleted file mode 100644
index ab54533..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/package-info.java
+++ /dev/null
@@ -1,34 +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.
- */
-/**
- * Provides a simple, powerful model for building both batch and
- * streaming parallel data processing
- * {@link com.google.cloud.dataflow.sdk.Pipeline}s.
- *
- * <p>To use the Google Cloud Dataflow SDK, you build a
- * {@link com.google.cloud.dataflow.sdk.Pipeline}, which manages a graph of
- * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s
- * and the {@link com.google.cloud.dataflow.sdk.values.PCollection}s that
- * the PTransforms consume and produce.
- *
- * <p>Each Pipeline has a
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to specify
- * where and how it should run after pipeline construction is complete.
- *
- */
-package com.google.cloud.dataflow.sdk;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorPipelineExtractor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorPipelineExtractor.java
deleted file mode 100644
index 3008c6c..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorPipelineExtractor.java
+++ /dev/null
@@ -1,98 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AggregatorRetriever;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.SetMultimap;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * Retrieves {@link Aggregator Aggregators} at each {@link ParDo} and returns a {@link Map} of
- * {@link Aggregator} to the {@link PTransform PTransforms} in which it is present.
- */
-public class AggregatorPipelineExtractor {
-  private final Pipeline pipeline;
-
-  /**
-   * Creates an {@code AggregatorPipelineExtractor} for the given {@link Pipeline}.
-   */
-  public AggregatorPipelineExtractor(Pipeline pipeline) {
-    this.pipeline = pipeline;
-  }
-
-  /**
-   * Returns a {@link Map} between each {@link Aggregator} in the {@link Pipeline} to the {@link
-   * PTransform PTransforms} in which it is used.
-   */
-  public Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> getAggregatorSteps() {
-    HashMultimap<Aggregator<?, ?>, PTransform<?, ?>> aggregatorSteps = HashMultimap.create();
-    pipeline.traverseTopologically(new AggregatorVisitor(aggregatorSteps));
-    return aggregatorSteps.asMap();
-  }
-
-  private static class AggregatorVisitor implements PipelineVisitor {
-    private final SetMultimap<Aggregator<?, ?>, PTransform<?, ?>> aggregatorSteps;
-
-    public AggregatorVisitor(SetMultimap<Aggregator<?, ?>, PTransform<?, ?>> aggregatorSteps) {
-      this.aggregatorSteps = aggregatorSteps;
-    }
-
-    @Override
-    public void enterCompositeTransform(TransformTreeNode node) {}
-
-    @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {}
-
-    @Override
-    public void visitTransform(TransformTreeNode node) {
-      PTransform<?, ?> transform = node.getTransform();
-      addStepToAggregators(transform, getAggregators(transform));
-    }
-
-    private Collection<Aggregator<?, ?>> getAggregators(PTransform<?, ?> transform) {
-      if (transform != null) {
-        if (transform instanceof ParDo.Bound) {
-          return AggregatorRetriever.getAggregators(((ParDo.Bound<?, ?>) transform).getFn());
-        } else if (transform instanceof ParDo.BoundMulti) {
-          return AggregatorRetriever.getAggregators(((ParDo.BoundMulti<?, ?>) transform).getFn());
-        }
-      }
-      return Collections.emptyList();
-    }
-
-    private void addStepToAggregators(
-        PTransform<?, ?> transform, Collection<Aggregator<?, ?>> aggregators) {
-      for (Aggregator<?, ?> aggregator : aggregators) {
-        aggregatorSteps.put(aggregator, transform);
-      }
-    }
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorRetrievalException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorRetrievalException.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorRetrievalException.java
deleted file mode 100644
index 2ed0afa..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorRetrievalException.java
+++ /dev/null
@@ -1,33 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-
-/**
- * Signals that an exception has occurred while retrieving {@link Aggregator}s.
- */
-public class AggregatorRetrievalException extends Exception {
-  /**
-   * Constructs a new {@code AggregatorRetrievalException} with the specified detail message and
-   * cause.
-   */
-  public AggregatorRetrievalException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorValues.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorValues.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorValues.java
deleted file mode 100644
index 8f26b36..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/AggregatorValues.java
+++ /dev/null
@@ -1,53 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * A collection of values associated with an {@link Aggregator}. Aggregators declared in a
- * {@link DoFn} are emitted on a per-{@code DoFn}-application basis.
- *
- * @param <T> the output type of the aggregator
- */
-public abstract class AggregatorValues<T> {
-  /**
-   * Get the values of the {@link Aggregator} at all steps it was used.
-   */
-  public Collection<T> getValues() {
-    return getValuesAtSteps().values();
-  }
-
-  /**
-   * Get the values of the {@link Aggregator} by the user name at each step it was used.
-   */
-  public abstract Map<String, T> getValuesAtSteps();
-
-  /**
-   * Get the total value of this {@link Aggregator} by applying the specified {@link CombineFn}.
-   */
-  public T getTotalValue(CombineFn<T, ?, T> combineFn) {
-    return combineFn.apply(getValues());
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java
deleted file mode 100644
index df596b2..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java
+++ /dev/null
@@ -1,56 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-
-/**
- * A {@link DirectPipeline} is a {@link Pipeline} that returns
- * {@link DirectPipelineRunner.EvaluationResults} when it is
- * {@link com.google.cloud.dataflow.sdk.Pipeline#run()}.
- */
-public class DirectPipeline extends Pipeline {
-
-  /**
-   * Creates and returns a new DirectPipeline instance for tests.
-   */
-  public static DirectPipeline createForTest() {
-    DirectPipelineRunner runner = DirectPipelineRunner.createForTest();
-    return new DirectPipeline(runner, runner.getPipelineOptions());
-  }
-
-  private DirectPipeline(DirectPipelineRunner runner, DirectPipelineOptions options) {
-    super(runner, options);
-  }
-
-  @Override
-  public DirectPipelineRunner.EvaluationResults run() {
-    return (DirectPipelineRunner.EvaluationResults) super.run();
-  }
-
-  @Override
-  public DirectPipelineRunner getRunner() {
-    return (DirectPipelineRunner) super.getRunner();
-  }
-
-  @Override
-  public String toString() {
-    return "DirectPipeline#" + hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java
deleted file mode 100644
index f4a5600..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java
+++ /dev/null
@@ -1,54 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for
- * the {@link DirectPipeline}.
- */
-public class DirectPipelineRegistrar {
-  private DirectPipelineRegistrar() { }
-
-  /**
-   * Register the {@link DirectPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class Runner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(DirectPipelineRunner.class);
-    }
-  }
-
-  /**
-   * Register the {@link DirectPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(DirectPipelineOptions.class);
-    }
-  }
-}



[08/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java
deleted file mode 100644
index 57e6116..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java
+++ /dev/null
@@ -1,1320 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.FileBasedSink;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
-import com.google.cloud.dataflow.sdk.util.AssignWindows;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunner;
-import com.google.cloud.dataflow.sdk.util.PerKeyCombineFnRunners;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-/**
- * Executes the operations in the pipeline directly, in this process, without
- * any optimization.  Useful for small local execution and tests.
- *
- * <p>Throws an exception from {@link #run} if execution fails.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code DirectPipelineRunner}, the Cloud Platform account that you configured with the
- * <a href="https://cloud.google.com/sdk/gcloud">gcloud</a> executable will need access to the
- * corresponding source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class DirectPipelineRunner
-    extends PipelineRunner<DirectPipelineRunner.EvaluationResults> {
-  private static final Logger LOG = LoggerFactory.getLogger(DirectPipelineRunner.class);
-
-  /**
-   * A source of random data, which can be seeded if determinism is desired.
-   */
-  private Random rand;
-
-  /**
-   * A map from PTransform class to the corresponding
-   * TransformEvaluator to use to evaluate that transform.
-   *
-   * <p>A static map that contains system-wide defaults.
-   */
-  private static Map<Class, TransformEvaluator> defaultTransformEvaluators =
-      new HashMap<>();
-
-  /**
-   * A map from PTransform class to the corresponding
-   * TransformEvaluator to use to evaluate that transform.
-   *
-   * <p>An instance map that contains bindings for this DirectPipelineRunner.
-   * Bindings in this map override those in the default map.
-   */
-  private Map<Class, TransformEvaluator> localTransformEvaluators =
-      new HashMap<>();
-
-  /**
-   * Records that instances of the specified PTransform class
-   * should be evaluated by default by the corresponding
-   * TransformEvaluator.
-   */
-  public static <TransformT extends PTransform<?, ?>>
-  void registerDefaultTransformEvaluator(
-      Class<TransformT> transformClass,
-      TransformEvaluator<? super TransformT> transformEvaluator) {
-    if (defaultTransformEvaluators.put(transformClass, transformEvaluator)
-        != null) {
-      throw new IllegalArgumentException(
-          "defining multiple evaluators for " + transformClass);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Records that instances of the specified PTransform class
-   * should be evaluated by the corresponding TransformEvaluator.
-   * Overrides any bindings specified by
-   * {@link #registerDefaultTransformEvaluator}.
-   */
-  public <TransformT extends PTransform<?, ?>>
-  void registerTransformEvaluator(
-      Class<TransformT> transformClass,
-      TransformEvaluator<TransformT> transformEvaluator) {
-    if (localTransformEvaluators.put(transformClass, transformEvaluator)
-        != null) {
-      throw new IllegalArgumentException(
-          "defining multiple evaluators for " + transformClass);
-    }
-  }
-
-  /**
-   * Returns the TransformEvaluator to use for instances of the
-   * specified PTransform class, or null if none registered.
-   */
-  public <TransformT extends PTransform<?, ?>>
-      TransformEvaluator<TransformT> getTransformEvaluator(Class<TransformT> transformClass) {
-    TransformEvaluator<TransformT> transformEvaluator =
-        localTransformEvaluators.get(transformClass);
-    if (transformEvaluator == null) {
-      transformEvaluator = defaultTransformEvaluators.get(transformClass);
-    }
-    return transformEvaluator;
-  }
-
-  /**
-   * Constructs a DirectPipelineRunner from the given options.
-   */
-  public static DirectPipelineRunner fromOptions(PipelineOptions options) {
-    DirectPipelineOptions directOptions =
-        PipelineOptionsValidator.validate(DirectPipelineOptions.class, options);
-    LOG.debug("Creating DirectPipelineRunner");
-    return new DirectPipelineRunner(directOptions);
-  }
-
-  /**
-   * Constructs a runner with default properties for testing.
-   *
-   * @return The newly created runner.
-   */
-  public static DirectPipelineRunner createForTest() {
-    DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class);
-    options.setStableUniqueNames(CheckEnabled.ERROR);
-    options.setGcpCredential(new TestCredential());
-    return new DirectPipelineRunner(options);
-  }
-
-  /**
-   * Enable runtime testing to verify that all functions and {@link Coder}
-   * instances can be serialized.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withSerializabilityTesting(boolean enable) {
-    this.testSerializability = enable;
-    return this;
-  }
-
-  /**
-   * Enable runtime testing to verify that all values can be encoded.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withEncodabilityTesting(boolean enable) {
-    this.testEncodability = enable;
-    return this;
-  }
-
-  /**
-   * Enable runtime testing to verify that functions do not depend on order
-   * of the elements.
-   *
-   * <p>This is accomplished by randomizing the order of elements.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withUnorderednessTesting(boolean enable) {
-    this.testUnorderedness = enable;
-    return this;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    if (transform instanceof Combine.GroupedValues) {
-      return (OutputT) applyTestCombine((Combine.GroupedValues) transform, (PCollection) input);
-    } else if (transform instanceof TextIO.Write.Bound) {
-      return (OutputT) applyTextIOWrite((TextIO.Write.Bound) transform, (PCollection<?>) input);
-    } else if (transform instanceof AvroIO.Write.Bound) {
-      return (OutputT) applyAvroIOWrite((AvroIO.Write.Bound) transform, (PCollection<?>) input);
-    } else if (transform instanceof GroupByKey) {
-      return (OutputT)
-          ((PCollection) input).apply(new GroupByKeyViaGroupByKeyOnly((GroupByKey) transform));
-    } else if (transform instanceof Window.Bound) {
-      return (OutputT)
-          ((PCollection) input).apply(new AssignWindowsAndSetStrategy((Window.Bound) transform));
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-  private <K, InputT, AccumT, OutputT> PCollection<KV<K, OutputT>> applyTestCombine(
-      Combine.GroupedValues<K, InputT, OutputT> transform,
-      PCollection<KV<K, Iterable<InputT>>> input) {
-
-    PCollection<KV<K, OutputT>> output = input
-        .apply(ParDo.of(TestCombineDoFn.create(transform, input, testSerializability, rand))
-            .withSideInputs(transform.getSideInputs()));
-
-    try {
-      output.setCoder(transform.getDefaultOutputCoder(input));
-    } catch (CannotProvideCoderException exc) {
-      // let coder inference occur later, if it can
-    }
-    return output;
-  }
-
-  private static class ElementProcessingOrderPartitionFn<T> implements PartitionFn<T> {
-    private int elementNumber;
-    @Override
-    public int partitionFor(T elem, int numPartitions) {
-      return elementNumber++ % numPartitions;
-    }
-  }
-
-  /**
-   * Applies TextIO.Write honoring user requested sharding controls (i.e. withNumShards)
-   * by applying a partition function based upon the number of shards the user requested.
-   */
-  private static class DirectTextIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final TextIO.Write.Bound<T> transform;
-
-    private DirectTextIOWrite(TextIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      checkState(transform.getNumShards() > 1,
-          "DirectTextIOWrite is expected to only be used when sharding controls are required.");
-
-      // Evenly distribute all the elements across the partitions.
-      PCollectionList<T> partitionedElements =
-          input.apply(Partition.of(transform.getNumShards(),
-                                   new ElementProcessingOrderPartitionFn<T>()));
-
-      // For each input PCollection partition, create a write transform that represents
-      // one of the specific shards.
-      for (int i = 0; i < transform.getNumShards(); ++i) {
-        /*
-         * This logic mirrors the file naming strategy within
-         * {@link FileBasedSink#generateDestinationFilenames()}
-         */
-        String outputFilename = IOChannelUtils.constructName(
-            transform.getFilenamePrefix(),
-            transform.getShardNameTemplate(),
-            getFileExtension(transform.getFilenameSuffix()),
-            i,
-            transform.getNumShards());
-
-        String transformName = String.format("%s(Shard:%s)", transform.getName(), i);
-        partitionedElements.get(i).apply(transformName,
-            transform.withNumShards(1).withShardNameTemplate("").withSuffix("").to(outputFilename));
-      }
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  /**
-   * Returns the file extension to be used. If the user did not request a file
-   * extension then this method returns the empty string. Otherwise this method
-   * adds a {@code "."} to the beginning of the users extension if one is not present.
-   *
-   * <p>This is copied from {@link FileBasedSink} to not expose it.
-   */
-  private static String getFileExtension(String usersExtension) {
-    if (usersExtension == null || usersExtension.isEmpty()) {
-      return "";
-    }
-    if (usersExtension.startsWith(".")) {
-      return usersExtension;
-    }
-    return "." + usersExtension;
-  }
-
-  /**
-   * Apply the override for TextIO.Write.Bound if the user requested sharding controls
-   * greater than one.
-   */
-  private <T> PDone applyTextIOWrite(TextIO.Write.Bound<T> transform, PCollection<T> input) {
-    if (transform.getNumShards() <= 1) {
-      // By default, the DirectPipelineRunner outputs to only 1 shard. Since the user never
-      // requested sharding controls greater than 1, we default to outputting to 1 file.
-      return super.apply(transform.withNumShards(1), input);
-    }
-    return input.apply(new DirectTextIOWrite<>(transform));
-  }
-
-  /**
-   * Applies AvroIO.Write honoring user requested sharding controls (i.e. withNumShards)
-   * by applying a partition function based upon the number of shards the user requested.
-   */
-  private static class DirectAvroIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final AvroIO.Write.Bound<T> transform;
-
-    private DirectAvroIOWrite(AvroIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      checkState(transform.getNumShards() > 1,
-          "DirectAvroIOWrite is expected to only be used when sharding controls are required.");
-
-      // Evenly distribute all the elements across the partitions.
-      PCollectionList<T> partitionedElements =
-          input.apply(Partition.of(transform.getNumShards(),
-                                   new ElementProcessingOrderPartitionFn<T>()));
-
-      // For each input PCollection partition, create a write transform that represents
-      // one of the specific shards.
-      for (int i = 0; i < transform.getNumShards(); ++i) {
-        /*
-         * This logic mirrors the file naming strategy within
-         * {@link FileBasedSink#generateDestinationFilenames()}
-         */
-        String outputFilename = IOChannelUtils.constructName(
-            transform.getFilenamePrefix(),
-            transform.getShardNameTemplate(),
-            getFileExtension(transform.getFilenameSuffix()),
-            i,
-            transform.getNumShards());
-
-        String transformName = String.format("%s(Shard:%s)", transform.getName(), i);
-        partitionedElements.get(i).apply(transformName,
-            transform.withNumShards(1).withShardNameTemplate("").withSuffix("").to(outputFilename));
-      }
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  private static class AssignWindowsAndSetStrategy<T, W extends BoundedWindow>
-      extends PTransform<PCollection<T>, PCollection<T>> {
-
-    private final Window.Bound<T> wrapped;
-
-    public AssignWindowsAndSetStrategy(Window.Bound<T> wrapped) {
-      this.wrapped = wrapped;
-    }
-
-    @Override
-    public PCollection<T> apply(PCollection<T> input) {
-      WindowingStrategy<?, ?> outputStrategy =
-          wrapped.getOutputStrategyInternal(input.getWindowingStrategy());
-
-      WindowFn<T, BoundedWindow> windowFn =
-          (WindowFn<T, BoundedWindow>) outputStrategy.getWindowFn();
-
-      // If the Window.Bound transform only changed parts other than the WindowFn, then
-      // we skip AssignWindows even though it should be harmless in a perfect world.
-      // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly
-      // crash if another GBK is performed without explicitly setting the WindowFn. So we skip
-      // AssignWindows in this case.
-      if (wrapped.getWindowFn() == null) {
-        return input.apply("Identity", ParDo.of(new IdentityFn<T>()))
-            .setWindowingStrategyInternal(outputStrategy);
-      } else {
-        return input
-            .apply("AssignWindows", new AssignWindows<T, BoundedWindow>(windowFn))
-            .setWindowingStrategyInternal(outputStrategy);
-      }
-    }
-  }
-
-  private static class IdentityFn<T> extends DoFn<T, T> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element());
-    }
-  }
-
-  /**
-   * Apply the override for AvroIO.Write.Bound if the user requested sharding controls
-   * greater than one.
-   */
-  private <T> PDone applyAvroIOWrite(AvroIO.Write.Bound<T> transform, PCollection<T> input) {
-    if (transform.getNumShards() <= 1) {
-      // By default, the DirectPipelineRunner outputs to only 1 shard. Since the user never
-      // requested sharding controls greater than 1, we default to outputting to 1 file.
-      return super.apply(transform.withNumShards(1), input);
-    }
-    return input.apply(new DirectAvroIOWrite<>(transform));
-  }
-
-  /**
-   * The implementation may split the {@link KeyedCombineFn} into ADD, MERGE and EXTRACT phases (
-   * see {@code com.google.cloud.dataflow.sdk.runners.worker.CombineValuesFn}). In order to emulate
-   * this for the {@link DirectPipelineRunner} and provide an experience closer to the service, go
-   * through heavy serializability checks for the equivalent of the results of the ADD phase, but
-   * after the {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} shuffle, and the MERGE
-   * phase. Doing these checks ensure that not only is the accumulator coder serializable, but
-   * the accumulator coder can actually serialize the data in question.
-   */
-  public static class TestCombineDoFn<K, InputT, AccumT, OutputT>
-      extends DoFn<KV<K, Iterable<InputT>>, KV<K, OutputT>> {
-    private final PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, OutputT> fnRunner;
-    private final Coder<AccumT> accumCoder;
-    private final boolean testSerializability;
-    private final Random rand;
-
-    public static <K, InputT, AccumT, OutputT> TestCombineDoFn<K, InputT, AccumT, OutputT> create(
-        Combine.GroupedValues<K, InputT, OutputT> transform,
-        PCollection<KV<K, Iterable<InputT>>> input,
-        boolean testSerializability,
-        Random rand) {
-
-      AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn = transform.getAppliedFn(
-          input.getPipeline().getCoderRegistry(), input.getCoder(), input.getWindowingStrategy());
-
-      return new TestCombineDoFn(
-          PerKeyCombineFnRunners.create(fn.getFn()),
-          fn.getAccumulatorCoder(),
-          testSerializability,
-          rand);
-    }
-
-    public TestCombineDoFn(
-        PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, OutputT> fnRunner,
-        Coder<AccumT> accumCoder,
-        boolean testSerializability,
-        Random rand) {
-      this.fnRunner = fnRunner;
-      this.accumCoder = accumCoder;
-      this.testSerializability = testSerializability;
-      this.rand = rand;
-
-      // Check that this does not crash, specifically to catch anonymous CustomCoder subclasses.
-      this.accumCoder.getEncodingId();
-    }
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      K key = c.element().getKey();
-      Iterable<InputT> values = c.element().getValue();
-      List<AccumT> groupedPostShuffle =
-          ensureSerializableByCoder(ListCoder.of(accumCoder),
-              addInputsRandomly(fnRunner, key, values, rand, c),
-              "After addInputs of KeyedCombineFn " + fnRunner.fn().toString());
-      AccumT merged =
-          ensureSerializableByCoder(accumCoder,
-            fnRunner.mergeAccumulators(key, groupedPostShuffle, c),
-            "After mergeAccumulators of KeyedCombineFn " + fnRunner.fn().toString());
-      // Note: The serializability of KV<K, OutputT> is ensured by the
-      // runner itself, since it's a transform output.
-      c.output(KV.of(key, fnRunner.extractOutput(key, merged, c)));
-    }
-
-    /**
-     * Create a random list of accumulators from the given list of values.
-     *
-     * <p>Visible for testing purposes only.
-     */
-    public static <K, AccumT, InputT> List<AccumT> addInputsRandomly(
-        PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, ?> fnRunner,
-        K key,
-        Iterable<InputT> values,
-        Random random,
-        DoFn<?, ?>.ProcessContext c) {
-      List<AccumT> out = new ArrayList<AccumT>();
-      int i = 0;
-      AccumT accumulator = fnRunner.createAccumulator(key, c);
-      boolean hasInput = false;
-
-      for (InputT value : values) {
-        accumulator = fnRunner.addInput(key, accumulator, value, c);
-        hasInput = true;
-
-        // For each index i, flip a 1/2^i weighted coin for whether to
-        // create a new accumulator after index i is added, i.e. [0]
-        // is guaranteed, [1] is an even 1/2, [2] is 1/4, etc. The
-        // goal is to partition the inputs into accumulators, and make
-        // the accumulators potentially lumpy.  Also compact about half
-        // of the accumulators.
-        if (i == 0 || random.nextInt(1 << Math.min(i, 30)) == 0) {
-          if (i % 2 == 0) {
-            accumulator = fnRunner.compact(key, accumulator, c);
-          }
-          out.add(accumulator);
-          accumulator = fnRunner.createAccumulator(key, c);
-          hasInput = false;
-        }
-        i++;
-      }
-      if (hasInput) {
-        out.add(accumulator);
-      }
-
-      Collections.shuffle(out, random);
-      return out;
-    }
-
-    public <T> T ensureSerializableByCoder(
-        Coder<T> coder, T value, String errorContext) {
-      if (testSerializability) {
-        return SerializableUtils.ensureSerializableByCoder(
-            coder, value, errorContext);
-      }
-      return value;
-    }
-  }
-
-  @Override
-  public EvaluationResults run(Pipeline pipeline) {
-    LOG.info("Executing pipeline using the DirectPipelineRunner.");
-
-    Evaluator evaluator = new Evaluator(rand);
-    evaluator.run(pipeline);
-
-    // Log all counter values for debugging purposes.
-    for (Counter counter : evaluator.getCounters()) {
-      LOG.info("Final aggregator value: {}", counter);
-    }
-
-    LOG.info("Pipeline execution complete.");
-
-    return evaluator;
-  }
-
-  /**
-   * An evaluator of a PTransform.
-   */
-  public interface TransformEvaluator<TransformT extends PTransform> {
-    public void evaluate(TransformT transform,
-                         EvaluationContext context);
-  }
-
-  /**
-   * The interface provided to registered callbacks for interacting
-   * with the {@code DirectPipelineRunner}, including reading and writing the
-   * values of {@link PCollection}s and {@link PCollectionView}s.
-   */
-  public interface EvaluationResults extends PipelineResult {
-    /**
-     * Retrieves the value of the given PCollection.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<T> getPCollection(PCollection<T> pc);
-
-    /**
-     * Retrieves the windowed value of the given PCollection.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<WindowedValue<T>> getPCollectionWindowedValues(PCollection<T> pc);
-
-    /**
-     * Retrieves the values of each PCollection in the given
-     * PCollectionList. Throws an exception if the PCollectionList's
-     * value hasn't already been set.
-     */
-    <T> List<List<T>> getPCollectionList(PCollectionList<T> pcs);
-
-    /**
-     * Retrieves the values indicated by the given {@link PCollectionView}.
-     * Note that within the {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context}
-     * implementation a {@link PCollectionView} should convert from this representation to a
-     * suitable side input value.
-     */
-    <T, WindowedT> Iterable<WindowedValue<?>> getPCollectionView(PCollectionView<T> view);
-  }
-
-  /**
-   * An immutable (value, timestamp) pair, along with other metadata necessary
-   * for the implementation of {@code DirectPipelineRunner}.
-   */
-  public static class ValueWithMetadata<V> {
-    /**
-     * Returns a new {@code ValueWithMetadata} with the {@code WindowedValue}.
-     * Key is null.
-     */
-    public static <V> ValueWithMetadata<V> of(WindowedValue<V> windowedValue) {
-      return new ValueWithMetadata<>(windowedValue, null);
-    }
-
-    /**
-     * Returns a new {@code ValueWithMetadata} with the implicit key associated
-     * with this value set.  The key is the last key grouped by in the chain of
-     * productions that produced this element.
-     * These keys are used internally by {@link DirectPipelineRunner} for keeping
-     * persisted state separate across keys.
-     */
-    public ValueWithMetadata<V> withKey(Object key) {
-      return new ValueWithMetadata<>(windowedValue, key);
-    }
-
-    /**
-     * Returns a new {@code ValueWithMetadata} that is a copy of this one, but with
-     * a different value.
-     */
-    public <T> ValueWithMetadata<T> withValue(T value) {
-      return new ValueWithMetadata(windowedValue.withValue(value), getKey());
-    }
-
-    /**
-     * Returns the {@code WindowedValue} associated with this element.
-     */
-    public WindowedValue<V> getWindowedValue() {
-      return windowedValue;
-    }
-
-    /**
-     * Returns the value associated with this element.
-     *
-     * @see #withValue
-     */
-    public V getValue() {
-      return windowedValue.getValue();
-    }
-
-    /**
-     * Returns the timestamp associated with this element.
-     */
-    public Instant getTimestamp() {
-      return windowedValue.getTimestamp();
-    }
-
-    /**
-     * Returns the collection of windows this element has been placed into.  May
-     * be null if the {@code PCollection} this element is in has not yet been
-     * windowed.
-     *
-     * @see #getWindows()
-     */
-    public Collection<? extends BoundedWindow> getWindows() {
-      return windowedValue.getWindows();
-    }
-
-
-    /**
-     * Returns the key associated with this element.  May be null if the
-     * {@code PCollection} this element is in is not keyed.
-     *
-     * @see #withKey
-     */
-    public Object getKey() {
-      return key;
-    }
-
-    ////////////////////////////////////////////////////////////////////////////
-
-  private final Object key;
-    private final WindowedValue<V> windowedValue;
-
-    private ValueWithMetadata(WindowedValue<V> windowedValue,
-                              Object key) {
-      this.windowedValue = windowedValue;
-      this.key = key;
-    }
-  }
-
-  /**
-   * The interface provided to registered callbacks for interacting
-   * with the {@code DirectPipelineRunner}, including reading and writing the
-   * values of {@link PCollection}s and {@link PCollectionView}s.
-   */
-  public interface EvaluationContext extends EvaluationResults {
-    /**
-     * Returns the configured pipeline options.
-     */
-    DirectPipelineOptions getPipelineOptions();
-
-    /**
-     * Returns the input of the currently being processed transform.
-     */
-    <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
-
-    /**
-     * Returns the output of the currently being processed transform.
-     */
-    <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
-
-    /**
-     * Sets the value of the given PCollection, where each element also has a timestamp
-     * and collection of windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollectionValuesWithMetadata(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements);
-
-    /**
-     * Sets the value of the given PCollection, where each element also has a timestamp
-     * and collection of windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollectionWindowedValue(PCollection<T> pc, List<WindowedValue<T>> elements);
-
-    /**
-     * Shorthand for setting the value of a PCollection where the elements do not have
-     * timestamps or windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollection(PCollection<T> pc, List<T> elements);
-
-    /**
-     * Retrieves the value of the given PCollection, along with element metadata
-     * such as timestamps and windows.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<ValueWithMetadata<T>> getPCollectionValuesWithMetadata(PCollection<T> pc);
-
-    /**
-     * Sets the value associated with the given {@link PCollectionView}.
-     * Throws an exception if the {@link PCollectionView}'s value has already been set.
-     */
-    <ElemT, T, WindowedT> void setPCollectionView(
-        PCollectionView<T> pc,
-        Iterable<WindowedValue<ElemT>> value);
-
-    /**
-     * Ensures that the element is encodable and decodable using the
-     * TypePValue's coder, by encoding it and decoding it, and
-     * returning the result.
-     */
-    <T> T ensureElementEncodable(TypedPValue<T> pvalue, T element);
-
-    /**
-     * If the evaluation context is testing unorderedness,
-     * randomly permutes the order of the elements, in a
-     * copy if !inPlaceAllowed, and returns the permuted list,
-     * otherwise returns the argument unchanged.
-     */
-    <T> List<T> randomizeIfUnordered(List<T> elements,
-                                     boolean inPlaceAllowed);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the argument function is serializable and deserializable
-     * by encoding it and then decoding it, and returning the result.
-     * Otherwise returns the argument unchanged.
-     */
-    <FunctionT extends Serializable> FunctionT ensureSerializable(FunctionT fn);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the argument Coder is serializable and deserializable
-     * by encoding it and then decoding it, and returning the result.
-     * Otherwise returns the argument unchanged.
-     */
-    <T> Coder<T> ensureCoderSerializable(Coder<T> coder);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the given data is serializable and deserializable with the
-     * given Coder by encoding it and then decoding it, and returning
-     * the result. Otherwise returns the argument unchanged.
-     *
-     * <p>Error context is prefixed to any thrown exceptions.
-     */
-    <T> T ensureSerializableByCoder(Coder<T> coder,
-                                    T data, String errorContext);
-
-    /**
-     * Returns a mutator, which can be used to add additional counters to
-     * this EvaluationContext.
-     */
-    CounterSet.AddCounterMutator getAddCounterMutator();
-
-    /**
-     * Gets the step name for this transform.
-     */
-    public String getStepName(PTransform<?, ?> transform);
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  class Evaluator implements PipelineVisitor, EvaluationContext {
-    /**
-     * A map from PTransform to the step name of that transform. This is the internal name for the
-     * transform (e.g. "s2").
-     */
-    private final Map<PTransform<?, ?>, String> stepNames = new HashMap<>();
-    private final Map<PValue, Object> store = new HashMap<>();
-    private final CounterSet counters = new CounterSet();
-    private AppliedPTransform<?, ?, ?> currentTransform;
-
-    private Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps = null;
-
-    /**
-     * A map from PTransform to the full name of that transform. This is the user name of the
-     * transform (e.g. "RemoveDuplicates/Combine/GroupByKey").
-     */
-    private final Map<PTransform<?, ?>, String> fullNames = new HashMap<>();
-
-    private Random rand;
-
-    public Evaluator() {
-      this(new Random());
-    }
-
-    public Evaluator(Random rand) {
-      this.rand = rand;
-    }
-
-    public void run(Pipeline pipeline) {
-      pipeline.traverseTopologically(this);
-      aggregatorSteps = new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
-    }
-
-    @Override
-    public DirectPipelineOptions getPipelineOptions() {
-      return options;
-    }
-
-    @Override
-    public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
-      checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-          "can only be called with current transform");
-      return (InputT) currentTransform.getInput();
-    }
-
-    @Override
-    public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
-      checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-          "can only be called with current transform");
-      return (OutputT) currentTransform.getOutput();
-    }
-
-    @Override
-    public void enterCompositeTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void visitTransform(TransformTreeNode node) {
-      PTransform<?, ?> transform = node.getTransform();
-      fullNames.put(transform, node.getFullName());
-      TransformEvaluator evaluator =
-          getTransformEvaluator(transform.getClass());
-      if (evaluator == null) {
-        throw new IllegalStateException(
-            "no evaluator registered for " + transform);
-      }
-      LOG.debug("Evaluating {}", transform);
-      currentTransform = AppliedPTransform.of(
-          node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
-      evaluator.evaluate(transform, this);
-      currentTransform = null;
-    }
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
-      LOG.debug("Checking evaluation of {}", value);
-      if (value.getProducingTransformInternal() == null) {
-        throw new RuntimeException(
-            "internal error: expecting a PValue " +
-            "to have a producingTransform");
-      }
-      if (!producer.isCompositeNode()) {
-        // Verify that primitive transform outputs are already computed.
-        getPValue(value);
-      }
-    }
-
-    /**
-     * Sets the value of the given PValue.
-     * Throws an exception if the PValue's value has already been set.
-     */
-    void setPValue(PValue pvalue, Object contents) {
-      if (store.containsKey(pvalue)) {
-        throw new IllegalStateException(
-            "internal error: setting the value of " + pvalue +
-            " more than once");
-      }
-      store.put(pvalue, contents);
-    }
-
-    /**
-     * Retrieves the value of the given PValue.
-     * Throws an exception if the PValue's value hasn't already been set.
-     */
-    Object getPValue(PValue pvalue) {
-      if (!store.containsKey(pvalue)) {
-        throw new IllegalStateException(
-            "internal error: getting the value of " + pvalue +
-            " before it has been computed");
-      }
-      return store.get(pvalue);
-    }
-
-    /**
-     * Convert a list of T to a list of {@code ValueWithMetadata<T>}, with a timestamp of 0
-     * and null windows.
-     */
-    <T> List<ValueWithMetadata<T>> toValueWithMetadata(List<T> values) {
-      List<ValueWithMetadata<T>> result = new ArrayList<>(values.size());
-      for (T value : values) {
-        result.add(ValueWithMetadata.of(WindowedValue.valueInGlobalWindow(value)));
-      }
-      return result;
-    }
-
-    /**
-     * Convert a list of {@code WindowedValue<T>} to a list of {@code ValueWithMetadata<T>}.
-     */
-    <T> List<ValueWithMetadata<T>> toValueWithMetadataFromWindowedValue(
-        List<WindowedValue<T>> values) {
-      List<ValueWithMetadata<T>> result = new ArrayList<>(values.size());
-      for (WindowedValue<T> value : values) {
-        result.add(ValueWithMetadata.of(value));
-      }
-      return result;
-    }
-
-    @Override
-    public <T> void setPCollection(PCollection<T> pc, List<T> elements) {
-      setPCollectionValuesWithMetadata(pc, toValueWithMetadata(elements));
-    }
-
-    @Override
-    public <T> void setPCollectionWindowedValue(
-        PCollection<T> pc, List<WindowedValue<T>> elements) {
-      setPCollectionValuesWithMetadata(pc, toValueWithMetadataFromWindowedValue(elements));
-    }
-
-    @Override
-    public <T> void setPCollectionValuesWithMetadata(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements) {
-      LOG.debug("Setting {} = {}", pc, elements);
-      ensurePCollectionEncodable(pc, elements);
-      setPValue(pc, elements);
-    }
-
-    @Override
-    public <ElemT, T, WindowedT> void setPCollectionView(
-        PCollectionView<T> view,
-        Iterable<WindowedValue<ElemT>> value) {
-      LOG.debug("Setting {} = {}", view, value);
-      setPValue(view, value);
-    }
-
-    /**
-     * Retrieves the value of the given {@link PCollection}.
-     * Throws an exception if the {@link PCollection}'s value hasn't already been set.
-     */
-    @Override
-    public <T> List<T> getPCollection(PCollection<T> pc) {
-      List<T> result = new ArrayList<>();
-      for (ValueWithMetadata<T> elem : getPCollectionValuesWithMetadata(pc)) {
-        result.add(elem.getValue());
-      }
-      return result;
-    }
-
-    @Override
-    public <T> List<WindowedValue<T>> getPCollectionWindowedValues(PCollection<T> pc) {
-      return Lists.transform(
-          getPCollectionValuesWithMetadata(pc),
-          new Function<ValueWithMetadata<T>, WindowedValue<T>>() {
-            @Override
-            public WindowedValue<T> apply(ValueWithMetadata<T> input) {
-              return input.getWindowedValue();
-            }});
-    }
-
-    @Override
-    public <T> List<ValueWithMetadata<T>> getPCollectionValuesWithMetadata(PCollection<T> pc) {
-      List<ValueWithMetadata<T>> elements = (List<ValueWithMetadata<T>>) getPValue(pc);
-      elements = randomizeIfUnordered(elements, false /* not inPlaceAllowed */);
-      LOG.debug("Getting {} = {}", pc, elements);
-      return elements;
-    }
-
-    @Override
-    public <T> List<List<T>> getPCollectionList(PCollectionList<T> pcs) {
-      List<List<T>> elementsList = new ArrayList<>();
-      for (PCollection<T> pc : pcs.getAll()) {
-        elementsList.add(getPCollection(pc));
-      }
-      return elementsList;
-    }
-
-    /**
-     * Retrieves the value indicated by the given {@link PCollectionView}.
-     * Note that within the {@link DoFnContext} a {@link PCollectionView}
-     * converts from this representation to a suitable side input value.
-     */
-    @Override
-    public <T, WindowedT> Iterable<WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
-      Iterable<WindowedValue<?>> value = (Iterable<WindowedValue<?>>) getPValue(view);
-      LOG.debug("Getting {} = {}", view, value);
-      return value;
-    }
-
-    /**
-     * If {@code testEncodability}, ensures that the {@link PCollection}'s coder and elements are
-     * encodable and decodable by encoding them and decoding them, and returning the result.
-     * Otherwise returns the argument elements.
-     */
-    <T> List<ValueWithMetadata<T>> ensurePCollectionEncodable(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements) {
-      ensureCoderSerializable(pc.getCoder());
-      if (!testEncodability) {
-        return elements;
-      }
-      List<ValueWithMetadata<T>> elementsCopy = new ArrayList<>(elements.size());
-      for (ValueWithMetadata<T> element : elements) {
-        elementsCopy.add(
-            element.withValue(ensureElementEncodable(pc, element.getValue())));
-      }
-      return elementsCopy;
-    }
-
-    @Override
-    public <T> T ensureElementEncodable(TypedPValue<T> pvalue, T element) {
-      return ensureSerializableByCoder(
-          pvalue.getCoder(), element, "Within " + pvalue.toString());
-    }
-
-    @Override
-    public <T> List<T> randomizeIfUnordered(List<T> elements,
-                                            boolean inPlaceAllowed) {
-      if (!testUnorderedness) {
-        return elements;
-      }
-      List<T> elementsCopy = new ArrayList<>(elements);
-      Collections.shuffle(elementsCopy, rand);
-      return elementsCopy;
-    }
-
-    @Override
-    public <FunctionT extends Serializable> FunctionT ensureSerializable(FunctionT fn) {
-      if (!testSerializability) {
-        return fn;
-      }
-      return SerializableUtils.ensureSerializable(fn);
-    }
-
-    @Override
-    public <T> Coder<T> ensureCoderSerializable(Coder<T> coder) {
-      if (testSerializability) {
-        SerializableUtils.ensureSerializable(coder);
-      }
-      return coder;
-    }
-
-    @Override
-    public <T> T ensureSerializableByCoder(
-        Coder<T> coder, T value, String errorContext) {
-      if (testSerializability) {
-        return SerializableUtils.ensureSerializableByCoder(
-            coder, value, errorContext);
-      }
-      return value;
-    }
-
-    @Override
-    public CounterSet.AddCounterMutator getAddCounterMutator() {
-      return counters.getAddCounterMutator();
-    }
-
-    @Override
-    public String getStepName(PTransform<?, ?> transform) {
-      String stepName = stepNames.get(transform);
-      if (stepName == null) {
-        stepName = "s" + (stepNames.size() + 1);
-        stepNames.put(transform, stepName);
-      }
-      return stepName;
-    }
-
-    /**
-     * Returns the CounterSet generated during evaluation, which includes
-     * user-defined Aggregators and may include system-defined counters.
-     */
-    public CounterSet getCounters() {
-      return counters;
-    }
-
-    /**
-     * Returns JobState.DONE in all situations. The Evaluator is not returned
-     * until the pipeline has been traversed, so it will either be returned
-     * after a successful run or the run call will terminate abnormally.
-     */
-    @Override
-    public State getState() {
-      return State.DONE;
-    }
-
-    @Override
-    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
-      Map<String, T> stepValues = new HashMap<>();
-      for (PTransform<?, ?> step : aggregatorSteps.get(aggregator)) {
-        String stepName = String.format("user-%s-%s", stepNames.get(step), aggregator.getName());
-        String fullName = fullNames.get(step);
-        Counter<?> counter = counters.getExistingCounter(stepName);
-        if (counter == null) {
-          throw new IllegalArgumentException(
-              "Aggregator " + aggregator + " is not used in this pipeline");
-        }
-        stepValues.put(fullName, (T) counter.getAggregate());
-      }
-      return new MapAggregatorValues<>(stepValues);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * The key by which GBK groups inputs - elements are grouped by the encoded form of the key,
-   * but the original key may be accessed as well.
-   */
-  private static class GroupingKey<K> {
-    private K key;
-    private byte[] encodedKey;
-
-    public GroupingKey(K key, byte[] encodedKey) {
-      this.key = key;
-      this.encodedKey = encodedKey;
-    }
-
-    public K getKey() {
-      return key;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof GroupingKey) {
-        GroupingKey<?> that = (GroupingKey<?>) o;
-        return Arrays.equals(this.encodedKey, that.encodedKey);
-      } else {
-        return false;
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      return Arrays.hashCode(encodedKey);
-    }
-  }
-
-  private final DirectPipelineOptions options;
-  private boolean testSerializability;
-  private boolean testEncodability;
-  private boolean testUnorderedness;
-
-  /** Returns a new DirectPipelineRunner. */
-  private DirectPipelineRunner(DirectPipelineOptions options) {
-    this.options = options;
-    // (Re-)register standard IO factories. Clobbers any prior credentials.
-    IOChannelUtils.registerStandardIOFactories(options);
-    long randomSeed;
-    if (options.getDirectPipelineRunnerRandomSeed() != null) {
-      randomSeed = options.getDirectPipelineRunnerRandomSeed();
-    } else {
-      randomSeed = new Random().nextLong();
-    }
-
-    LOG.debug("DirectPipelineRunner using random seed {}.", randomSeed);
-    rand = new Random(randomSeed);
-
-    testSerializability = options.isTestSerializability();
-    testEncodability = options.isTestEncodability();
-    testUnorderedness = options.isTestUnorderedness();
-  }
-
-  /**
-   * Get the options used in this {@link Pipeline}.
-   */
-  public DirectPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public String toString() {
-    return "DirectPipelineRunner#" + hashCode();
-  }
-
-  public static <K, V> void evaluateGroupByKeyOnly(
-      GroupByKeyOnly<K, V> transform,
-      EvaluationContext context) {
-    PCollection<KV<K, V>> input = context.getInput(transform);
-
-    List<ValueWithMetadata<KV<K, V>>> inputElems =
-        context.getPCollectionValuesWithMetadata(input);
-
-    Coder<K> keyCoder = GroupByKey.getKeyCoder(input.getCoder());
-
-    Map<GroupingKey<K>, List<V>> groupingMap = new HashMap<>();
-
-    for (ValueWithMetadata<KV<K, V>> elem : inputElems) {
-      K key = elem.getValue().getKey();
-      V value = elem.getValue().getValue();
-      byte[] encodedKey;
-      try {
-        encodedKey = encodeToByteArray(keyCoder, key);
-      } catch (CoderException exn) {
-        // TODO: Put in better element printing:
-        // truncate if too long.
-        throw new IllegalArgumentException(
-            "unable to encode key " + key + " of input to " + transform +
-            " using " + keyCoder,
-            exn);
-      }
-      GroupingKey<K> groupingKey =
-          new GroupingKey<>(key, encodedKey);
-      List<V> values = groupingMap.get(groupingKey);
-      if (values == null) {
-        values = new ArrayList<V>();
-        groupingMap.put(groupingKey, values);
-      }
-      values.add(value);
-    }
-
-    List<ValueWithMetadata<KV<K, Iterable<V>>>> outputElems =
-        new ArrayList<>();
-    for (Map.Entry<GroupingKey<K>, List<V>> entry : groupingMap.entrySet()) {
-      GroupingKey<K> groupingKey = entry.getKey();
-      K key = groupingKey.getKey();
-      List<V> values = entry.getValue();
-      values = context.randomizeIfUnordered(values, true /* inPlaceAllowed */);
-      outputElems.add(ValueWithMetadata
-                      .of(WindowedValue.valueInEmptyWindows(KV.<K, Iterable<V>>of(key, values)))
-                      .withKey(key));
-    }
-
-    context.setPCollectionValuesWithMetadata(context.getOutput(transform),
-                                             outputElems);
-  }
-
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  public
-  static <K, V> void registerGroupByKeyOnly() {
-    registerDefaultTransformEvaluator(
-        GroupByKeyOnly.class,
-        new TransformEvaluator<GroupByKeyOnly>() {
-          @Override
-          public void evaluate(
-              GroupByKeyOnly transform,
-              EvaluationContext context) {
-            evaluateGroupByKeyOnly(transform, context);
-          }
-        });
-  }
-
-  static {
-    registerGroupByKeyOnly();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java
deleted file mode 100644
index 7f25183..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java
+++ /dev/null
@@ -1,77 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.base.Preconditions;
-
-/**
- * A {@link PipelineRunner} can execute, translate, or otherwise process a
- * {@link Pipeline}.
- *
- * @param <ResultT> the type of the result of {@link #run}.
- */
-public abstract class PipelineRunner<ResultT extends PipelineResult> {
-
-  /**
-   * Constructs a runner from the provided options.
-   *
-   * @return The newly created runner.
-   */
-  public static PipelineRunner<? extends PipelineResult> fromOptions(PipelineOptions options) {
-    GcsOptions gcsOptions = PipelineOptionsValidator.validate(GcsOptions.class, options);
-    Preconditions.checkNotNull(options);
-
-    // (Re-)register standard IO factories. Clobbers any prior credentials.
-    IOChannelUtils.registerStandardIOFactories(gcsOptions);
-
-    @SuppressWarnings("unchecked")
-    PipelineRunner<? extends PipelineResult> result =
-        InstanceBuilder.ofType(PipelineRunner.class)
-        .fromClass(options.getRunner())
-        .fromFactoryMethod("fromOptions")
-        .withArg(PipelineOptions.class, options)
-        .build();
-    return result;
-  }
-
-  /**
-   * Processes the given Pipeline, returning the results.
-   */
-  public abstract ResultT run(Pipeline pipeline);
-
-  /**
-   * Applies a transform to the given input, returning the output.
-   *
-   * <p>The default implementation calls PTransform.apply(input), but can be overridden
-   * to customize behavior for a particular runner.
-   */
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    return transform.apply(input);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java
deleted file mode 100644
index 1acd71f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java
+++ /dev/null
@@ -1,41 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.auto.service.AutoService;
-import java.util.ServiceLoader;
-
-/**
- * {@link PipelineRunner} creators have the ability to automatically have their
- * {@link PipelineRunner} registered with this SDK by creating a {@link ServiceLoader} entry
- * and a concrete implementation of this interface.
- *
- * <p>Note that automatic registration of any
- * {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} requires users
- * conform to the limit that each {@link PipelineRunner}'s
- * {@link Class#getSimpleName() simple name} must be unique.
- *
- * <p>It is optional but recommended to use one of the many build time tools such as
- * {@link AutoService} to generate the necessary META-INF files automatically.
- */
-public interface PipelineRunnerRegistrar {
-  /**
-   * Get the set of {@link PipelineRunner PipelineRunners} to register.
-   */
-  public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java
deleted file mode 100644
index ee9e8df..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java
+++ /dev/null
@@ -1,55 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides a simple {@link com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor}
- * that records the transformation tree.
- *
- * <p>Provided for internal unit tests.
- */
-public class RecordingPipelineVisitor implements Pipeline.PipelineVisitor {
-
-  public final List<PTransform<?, ?>> transforms = new ArrayList<>();
-  public final List<PValue> values = new ArrayList<>();
-
-  @Override
-  public void enterCompositeTransform(TransformTreeNode node) {
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
-  }
-
-  @Override
-  public void visitTransform(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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java
deleted file mode 100644
index 23687c4..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java
+++ /dev/null
@@ -1,105 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Preconditions;
-
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Captures information about a collection of transformations and their
- * associated {@link PValue}s.
- */
-public class TransformHierarchy {
-  private final Deque<TransformTreeNode> transformStack = new LinkedList<>();
-  private final Map<PInput, TransformTreeNode> producingTransformNode = new HashMap<>();
-
-  /**
-   * Create a {@code TransformHierarchy} containing a root node.
-   */
-  public TransformHierarchy() {
-    // First element in the stack is the root node, holding all child nodes.
-    transformStack.add(new TransformTreeNode(null, null, "", null));
-  }
-
-  /**
-   * Returns the last TransformTreeNode on the stack.
-   */
-  public TransformTreeNode getCurrent() {
-    return transformStack.peek();
-  }
-
-  /**
-   * Add a TransformTreeNode to the stack.
-   */
-  public void pushNode(TransformTreeNode current) {
-    transformStack.push(current);
-  }
-
-  /**
-   * Removes the last TransformTreeNode from the stack.
-   */
-  public void popNode() {
-    transformStack.pop();
-    Preconditions.checkState(!transformStack.isEmpty());
-  }
-
-  /**
-   * Adds an input to the given node.
-   *
-   * <p>This forces the producing node to be finished.
-   */
-  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);
-      }
-
-      producer.finishSpecifying();
-      node.addInputProducer(i, producer);
-    }
-  }
-
-  /**
-   * Sets the output of a transform node.
-   */
-  public void setOutput(TransformTreeNode producer, POutput output) {
-    producer.setOutput(output);
-
-    for (PValue o : output.expand()) {
-      producingTransformNode.put(o, producer);
-    }
-  }
-
-  /**
-   * Visits all nodes in the transform hierarchy, in transitive order.
-   */
-  public void visit(Pipeline.PipelineVisitor visitor,
-                    Set<PValue> visitedNodes) {
-    transformStack.peekFirst().visit(visitor, visitedNodes);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java
deleted file mode 100644
index 60d308f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java
+++ /dev/null
@@ -1,253 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Preconditions;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * Provides internal tracking of transform relationships with helper methods
- * for initialization and ordered visitation.
- */
-public class TransformTreeNode {
-  private final TransformTreeNode enclosingNode;
-
-  // The PTransform for this node, which may be a composite PTransform.
-  // The root of a TransformHierarchy is represented as a TransformTreeNode
-  // with a null transform field.
-  private final PTransform<?, ?> transform;
-
-  private final String fullName;
-
-  // Nodes for sub-transforms of a composite transform.
-  private final Collection<TransformTreeNode> parts = new ArrayList<>();
-
-  // Inputs to the transform, in expanded form and mapped to the producer
-  // of the input.
-  private final Map<PValue, TransformTreeNode> inputs = new HashMap<>();
-
-  // Input to the transform, in unexpanded form.
-  private final PInput input;
-
-  // TODO: track which outputs need to be exported to parent.
-  // Output of the transform, in unexpanded form.
-  private POutput output;
-
-  private boolean finishedSpecifying = false;
-
-  /**
-   * Creates a new TransformTreeNode with the given parent and transform.
-   *
-   * <p>EnclosingNode and transform may both be null for
-   * a root-level node, which holds all other nodes.
-   *
-   * @param enclosingNode the composite node containing this node
-   * @param transform the PTransform tracked by this node
-   * @param fullName the fully qualified name of the transform
-   * @param input the unexpanded input to the transform
-   */
-  public TransformTreeNode(@Nullable TransformTreeNode enclosingNode,
-                           @Nullable PTransform<?, ?> transform,
-                           String fullName,
-                           @Nullable PInput input) {
-    this.enclosingNode = enclosingNode;
-    this.transform = transform;
-    Preconditions.checkArgument((enclosingNode == null && transform == null)
-        || (enclosingNode != null && transform != null),
-        "EnclosingNode and transform must both be specified, or both be null");
-    this.fullName = fullName;
-    this.input = input;
-  }
-
-  /**
-   * Returns the transform associated with this transform node.
-   */
-  public PTransform<?, ?> getTransform() {
-    return transform;
-  }
-
-  /**
-   * Returns the enclosing composite transform node, or null if there is none.
-   */
-  public TransformTreeNode getEnclosingNode() {
-    return enclosingNode;
-  }
-
-  /**
-   * Adds a composite operation to the transform node.
-   *
-   * <p>As soon as a node is added, the transform node is considered a
-   * composite operation instead of a primitive transform.
-   */
-  public void addComposite(TransformTreeNode node) {
-    parts.add(node);
-  }
-
-  /**
-   * Returns true if this node represents a composite transform that does not perform
-   * processing of its own, but merely encapsulates a sub-pipeline (which may be empty).
-   *
-   * <p>Note that a node may be composite with no sub-transforms if it  returns its input directly
-   * extracts a component of a tuple, or other operations that occur at pipeline assembly time.
-   */
-  public boolean isCompositeNode() {
-    return !parts.isEmpty() || returnsOthersOutput() || isRootNode();
-  }
-
-  private boolean returnsOthersOutput() {
-    PTransform<?, ?> transform = getTransform();
-    for (PValue output : getExpandedOutputs()) {
-      if (!output.getProducingTransformInternal().getTransform().equals(transform)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public boolean isRootNode() {
-    return transform == null;
-  }
-
-  public String getFullName() {
-    return fullName;
-  }
-
-  /**
-   * Adds an input to the transform node.
-   */
-  public void addInputProducer(PValue expandedInput, TransformTreeNode producer) {
-    Preconditions.checkState(!finishedSpecifying);
-    inputs.put(expandedInput, producer);
-  }
-
-  /**
-   * Returns the transform input, in unexpanded form.
-   */
-  public PInput getInput() {
-    return input;
-  }
-
-  /**
-   * Returns a mapping of inputs to the producing nodes for all inputs to
-   * the transform.
-   */
-  public Map<PValue, TransformTreeNode> getInputs() {
-    return Collections.unmodifiableMap(inputs);
-  }
-
-  /**
-   * Adds an output to the transform node.
-   */
-  public void setOutput(POutput output) {
-    Preconditions.checkState(!finishedSpecifying);
-    Preconditions.checkState(this.output == null);
-    this.output = output;
-  }
-
-  /**
-   * Returns the transform output, in unexpanded form.
-   */
-  public POutput getOutput() {
-    return output;
-  }
-
-  /**
-   * Returns the transform outputs, in expanded form.
-   */
-  public Collection<? extends PValue> getExpandedOutputs() {
-    if (output != null) {
-      return output.expand();
-    } else {
-      return Collections.emptyList();
-    }
-  }
-
-  /**
-   * Visit the transform node.
-   *
-   * <p>Provides an ordered visit of the input values, the primitive
-   * transform (or child nodes for composite transforms), then the
-   * output values.
-   */
-  public void visit(Pipeline.PipelineVisitor visitor,
-                    Set<PValue> visitedValues) {
-    if (!finishedSpecifying) {
-      finishSpecifying();
-    }
-
-    // Visit inputs.
-    for (Map.Entry<PValue, TransformTreeNode> entry : inputs.entrySet()) {
-      if (visitedValues.add(entry.getKey())) {
-        visitor.visitValue(entry.getKey(), entry.getValue());
-      }
-    }
-
-    if (isCompositeNode()) {
-      visitor.enterCompositeTransform(this);
-      for (TransformTreeNode child : parts) {
-        child.visit(visitor, visitedValues);
-      }
-      visitor.leaveCompositeTransform(this);
-    } else {
-      visitor.visitTransform(this);
-    }
-
-    // Visit outputs.
-    for (PValue pValue : getExpandedOutputs()) {
-      if (visitedValues.add(pValue)) {
-        visitor.visitValue(pValue, this);
-      }
-    }
-  }
-
-  /**
-   * Finish specifying a transform.
-   *
-   * <p>All inputs are finished first, then the transform, then
-   * all outputs.
-   */
-  public void finishSpecifying() {
-    if (finishedSpecifying) {
-      return;
-    }
-    finishedSpecifying = true;
-
-    for (TransformTreeNode input : inputs.values()) {
-      if (input != null) {
-        input.finishSpecifying();
-      }
-    }
-
-    if (output != null) {
-      output.finishSpecifyingOutput();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java
deleted file mode 100644
index 6342725..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java
+++ /dev/null
@@ -1,38 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-/**
- * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.
- */
-abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
-  @Override
-  public void beforeElement(WindowedValue<T> element) {}
-
-  @Override
-  public void afterElement(WindowedValue<T> element) {}
-
-  @Override
-  public void afterFinish(
-      CommittedBundle<T> input,
-      InProcessTransformResult result,
-      Iterable<? extends CommittedBundle<?>> outputs) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AvroIOShardedWriteFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
deleted file mode 100644
index 49576e5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
deleted file mode 100644
index f034e2f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,156 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Bounded Read.Bounded} primitive {@link PTransform}.
- */
-final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
-  /*
-   * An evaluator for a Source is stateful, to ensure data is not read multiple times.
-   * Evaluators are cached here to ensure that the reader is not restarted if the evaluator is
-   * retriggered.
-   */
-  private final ConcurrentMap<EvaluatorKey, Queue<? extends BoundedReadEvaluator<?>>>
-      sourceEvaluators = new ConcurrentHashMap<>();
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws IOException {
-    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
-  }
-
-  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
-      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    BoundedReadEvaluator<?> evaluator =
-        getTransformEvaluatorQueue(transform, evaluationContext).poll();
-    if (evaluator == null) {
-      return EmptyTransformEvaluator.create(transform);
-    }
-    return evaluator;
-  }
-
-  /**
-   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
-   * provided application of {@link Bounded Read.Bounded}, initializing it if required.
-   *
-   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
-   * already done so.
-   */
-  @SuppressWarnings("unchecked")
-  private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
-      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    // Key by the application and the context the evaluation is occurring in (which call to
-    // Pipeline#run).
-    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
-    Queue<BoundedReadEvaluator<OutputT>> evaluatorQueue =
-        (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-    if (evaluatorQueue == null) {
-      evaluatorQueue = new ConcurrentLinkedQueue<>();
-      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
-        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
-        // factory for this transform
-        BoundedSource<OutputT> source = transform.getTransform().getSource();
-        BoundedReadEvaluator<OutputT> evaluator =
-            new BoundedReadEvaluator<OutputT>(transform, evaluationContext, source);
-        evaluatorQueue.offer(evaluator);
-      } else {
-        // otherwise return the existing Queue that arrived before us
-        evaluatorQueue = (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-      }
-    }
-    return evaluatorQueue;
-  }
-
-  /**
-   * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource},
-   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
-   * creates the {@link BoundedReader} and consumes all available input.
-   *
-   * <p>A {@link BoundedReadEvaluator} should only be created once per {@link BoundedSource}, and
-   * each evaluator should only be called once per evaluation of the pipeline. Otherwise, the source
-   * may produce duplicate elements.
-   */
-  private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
-    private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
-    /**
-     * The source being read from by this {@link BoundedReadEvaluator}. This may not be the same
-     * as the source derived from {@link #transform} due to splitting.
-     */
-    private BoundedSource<OutputT> source;
-
-    public BoundedReadEvaluator(
-        AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
-        BoundedSource<OutputT> source) {
-      this.transform = transform;
-      this.evaluationContext = evaluationContext;
-      this.source = source;
-    }
-
-    @Override
-    public void processElement(WindowedValue<Object> element) {}
-
-    @Override
-    public InProcessTransformResult finishBundle() throws IOException {
-      try (final BoundedReader<OutputT> reader =
-              source.createReader(evaluationContext.getPipelineOptions());) {
-        boolean contentsRemaining = reader.start();
-        UncommittedBundle<OutputT> output =
-            evaluationContext.createRootBundle(transform.getOutput());
-        while (contentsRemaining) {
-          output.add(
-              WindowedValue.timestampedValueInGlobalWindow(
-                  reader.getCurrent(), reader.getCurrentTimestamp()));
-          contentsRemaining = reader.advance();
-        }
-        reader.close();
-        return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MAX_VALUE)
-            .addOutput(output)
-            .build();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BundleFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BundleFactory.java
deleted file mode 100644
index cb8a369..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/BundleFactory.java
+++ /dev/null
@@ -1,50 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-/**
- * A factory that creates {@link UncommittedBundle UncommittedBundles}.
- */
-public interface BundleFactory {
-  /**
-   * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle belong to
-   * the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output);
-
-  /**
-   * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output);
-
-  /**
-   * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output);
-}
-



[21/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java
deleted file mode 100644
index a43e95e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java
+++ /dev/null
@@ -1,138 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
-import com.google.common.collect.Lists;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * An abstract base class for writing a {@link Coder} class that encodes itself via Java
- * serialization.
- *
- * <p>To complete an implementation, subclasses must implement {@link Coder#encode}
- * and {@link Coder#decode} methods. Anonymous subclasses must furthermore override
- * {@link #getEncodingId}.
- *
- * <p>Not to be confused with {@link SerializableCoder} that encodes objects that implement the
- * {@link Serializable} interface.
- *
- * @param <T> the type of elements handled by this coder
- */
-public abstract class CustomCoder<T> extends AtomicCoder<T>
-    implements Serializable {
-  @JsonCreator
-  public static CustomCoder<?> of(
-      // N.B. typeId is a required parameter here, since a field named "@type"
-      // is presented to the deserializer as an input.
-      //
-      // If this method did not consume the field, Jackson2 would observe an
-      // unconsumed field and a returned value of a derived type.  So Jackson2
-      // would attempt to update the returned value with the unconsumed field
-      // data, The standard JsonDeserializer does not implement a mechanism for
-      // updating constructed values, so it would throw an exception, causing
-      // deserialization to fail.
-      @JsonProperty(value = "@type", required = false) String typeId,
-      @JsonProperty(value = "encoding_id", required = false) String encodingId,
-      @JsonProperty("type") String type,
-      @JsonProperty("serialized_coder") String serializedCoder) {
-    return (CustomCoder<?>) SerializableUtils.deserializeFromByteArray(
-        StringUtils.jsonStringToByteArray(serializedCoder),
-        type);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return A thin {@link CloudObject} wrapping of the Java serialization of {@code this}.
-   */
-  @Override
-  public CloudObject asCloudObject() {
-    // N.B. We use the CustomCoder class, not the derived class, since during
-    // deserialization we will be using the CustomCoder's static factory method
-    // to construct an instance of the derived class.
-    CloudObject result = CloudObject.forClass(CustomCoder.class);
-    addString(result, "type", getClass().getName());
-    addString(result, "serialized_coder",
-        StringUtils.byteArrayToJsonString(
-            SerializableUtils.serializeToByteArray(this)));
-
-    String encodingId = getEncodingId();
-    checkNotNull(encodingId, "Coder.getEncodingId() must not return null.");
-    if (!encodingId.isEmpty()) {
-      addString(result, PropertyNames.ENCODING_ID, encodingId);
-    }
-
-    Collection<String> allowedEncodings = getAllowedEncodings();
-    if (!allowedEncodings.isEmpty()) {
-      addStringList(result, PropertyNames.ALLOWED_ENCODINGS, Lists.newArrayList(allowedEncodings));
-    }
-
-    return result;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException a {@link CustomCoder} is presumed
-   * nondeterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "CustomCoder implementations must override verifyDeterministic,"
-        + " or they are presumed nondeterministic.");
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return The canonical class name for this coder. For stable data formats that are independent
-   *         of class name, it is recommended to override this method.
-   *
-   * @throws UnsupportedOperationException when an anonymous class is used, since they do not have
-   *         a stable canonical class name.
-   */
-  @Override
-  public String getEncodingId() {
-    if (getClass().isAnonymousClass()) {
-      throw new UnsupportedOperationException(
-          String.format("Anonymous CustomCoder subclass %s must override getEncodingId()."
-              + " Otherwise, convert to a named class and getEncodingId() will be automatically"
-              + " generated from the fully qualified class name.",
-              getClass()));
-    }
-    return getClass().getCanonicalName();
-  }
-
-  // This coder inherits isRegisterByteSizeObserverCheap,
-  // getEncodedElementByteSize and registerByteSizeObserver
-  // from StandardCoder. Override if we can do better.
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java
deleted file mode 100644
index b277166..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java
+++ /dev/null
@@ -1,67 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.lang.annotation.Documented;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * The {@link DefaultCoder} annotation
- * specifies a default {@link Coder} class to handle encoding and decoding
- * instances of the annotated class.
- *
- * <p>The specified {@link Coder} must satisfy the requirements of
- * {@link CoderProviders#fromStaticMethods}. Two classes provided by the SDK that
- * are intended for use with this annotation include {@link SerializableCoder}
- * and {@link AvroCoder}.
- *
- * <p>To configure the use of Java serialization as the default
- * for a class, annotate the class to use
- * {@link SerializableCoder} as follows:
- *
- * <pre><code>{@literal @}DefaultCoder(SerializableCoder.class)
- * public class MyCustomDataType implements Serializable {
- *   // ...
- * }</code></pre>
- *
- * <p>Similarly, to configure the use of
- * {@link AvroCoder} as the default:
- * <pre><code>{@literal @}DefaultCoder(AvroCoder.class)
- * public class MyCustomDataType {
- *   public MyCustomDataType() {}  // Avro requires an empty constructor.
- *   // ...
- * }</code></pre>
- *
- * <p>Coders specified explicitly via
- * {@link PCollection#setCoder}
- * take precedence, followed by Coders registered at runtime via
- * {@link CoderRegistry#registerCoder}. See {@link CoderRegistry} for a more detailed discussion
- * of the precedence rules.
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-@SuppressWarnings("rawtypes")
-public @interface DefaultCoder {
-  Class<? extends Coder> value();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java
deleted file mode 100644
index d1df083..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java
+++ /dev/null
@@ -1,165 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.common.collect.Lists;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * A {@code DelegateCoder<T, IntermediateT>} wraps a {@link Coder} for {@code IntermediateT} and
- * encodes/decodes values of type {@code T} by converting
- * to/from {@code IntermediateT} and then encoding/decoding using the underlying
- * {@code Coder<IntermediateT>}.
- *
- * <p>The conversions from {@code T} to {@code IntermediateT} and vice versa
- * must be supplied as {@link CodingFunction}, a serializable
- * function that may throw any {@code Exception}. If a thrown
- * exception is an instance of {@link CoderException} or
- * {@link IOException}, it will be re-thrown, otherwise it will be wrapped as
- * a {@link CoderException}.
- *
- * @param <T> The type of objects coded by this Coder.
- * @param <IntermediateT> The type of objects a {@code T} will be converted to for coding.
- */
-public class DelegateCoder<T, IntermediateT> extends CustomCoder<T> {
-  /**
-   * A {@link DelegateCoder.CodingFunction CodingFunction&lt;InputT, OutputT&gt;} is a serializable
-   * function from {@code InputT} to {@code OutputT} that may throw any {@link Exception}.
-   */
-  public static interface CodingFunction<InputT, OutputT> extends Serializable {
-     public abstract OutputT apply(InputT input) throws Exception;
-  }
-
-  public static <T, IntermediateT> DelegateCoder<T, IntermediateT> of(Coder<IntermediateT> coder,
-      CodingFunction<T, IntermediateT> toFn,
-      CodingFunction<IntermediateT, T> fromFn) {
-    return new DelegateCoder<T, IntermediateT>(coder, toFn, fromFn);
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context)
-      throws CoderException, IOException {
-    coder.encode(applyAndWrapExceptions(toFn, value), outStream, context);
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws CoderException, IOException {
-    return applyAndWrapExceptions(fromFn, coder.decode(inStream, context));
-  }
-
-  /**
-   * Returns the coder used to encode/decode the intermediate values produced/consumed by the
-   * coding functions of this {@code DelegateCoder}.
-   */
-  public Coder<IntermediateT> getCoder() {
-    return coder;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException when the underlying coder's {@code verifyDeterministic()}
-   *         throws a {@link Coder.NonDeterministicException}. For this to be safe, the
-   *         intermediate {@code CodingFunction<T, IntermediateT>} must also be deterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    coder.verifyDeterministic();
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return a structural for a value of type {@code T} obtained by first converting to
-   *         {@code IntermediateT} and then obtaining a structural value according to the underlying
-   *         coder.
-   */
-  @Override
-  public Object structuralValue(T value) throws Exception {
-    return coder.structuralValue(toFn.apply(value));
-  }
-
-  @Override
-  public String toString() {
-    return "DelegateCoder(" + coder + ")";
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return a {@link String} composed from the underlying coder class name and its encoding id.
-   *         Note that this omits any description of the coding functions. These should be modified
-   *         with care.
-   */
-  @Override
-  public String getEncodingId() {
-    return delegateEncodingId(coder.getClass(), coder.getEncodingId());
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return allowed encodings which are composed from the underlying coder class and its allowed
-   *         encoding ids. Note that this omits any description of the coding functions. These
-   *         should be modified with care.
-   */
-  @Override
-  public Collection<String> getAllowedEncodings() {
-    List<String> allowedEncodings = Lists.newArrayList();
-    for (String allowedEncoding : coder.getAllowedEncodings()) {
-      allowedEncodings.add(delegateEncodingId(coder.getClass(), allowedEncoding));
-    }
-    return allowedEncodings;
-  }
-
-  private String delegateEncodingId(Class<?> delegateClass, String encodingId) {
-    return String.format("%s:%s", delegateClass.getName(), encodingId);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private <InputT, OutputT> OutputT applyAndWrapExceptions(
-      CodingFunction<InputT, OutputT> fn,
-      InputT input) throws CoderException, IOException {
-    try {
-      return fn.apply(input);
-    } catch (IOException exc) {
-      throw exc;
-    } catch (Exception exc) {
-      throw new CoderException(exc);
-    }
-  }
-
-  private final Coder<IntermediateT> coder;
-  private final CodingFunction<T, IntermediateT> toFn;
-  private final CodingFunction<IntermediateT, T> fromFn;
-
-  protected DelegateCoder(Coder<IntermediateT> coder,
-      CodingFunction<T, IntermediateT> toFn,
-      CodingFunction<IntermediateT, T> fromFn) {
-    this.coder = coder;
-    this.fromFn = fromFn;
-    this.toFn = toFn;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java
deleted file mode 100644
index 6dc2506..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java
+++ /dev/null
@@ -1,39 +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 com.google.cloud.dataflow.sdk.coders;
-
-/**
- * A {@link DeterministicStandardCoder} is a {@link StandardCoder} that is
- * deterministic, in the sense that for objects considered equal
- * according to {@link Object#equals(Object)}, the encoded bytes are
- * also equal.
- *
- * @param <T> the type of the values being transcoded
- */
-public abstract class DeterministicStandardCoder<T> extends StandardCoder<T> {
-  protected DeterministicStandardCoder() {}
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException never, unless overridden. A
-   * {@link DeterministicStandardCoder} is presumed deterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException { }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java
deleted file mode 100644
index 44cade6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java
+++ /dev/null
@@ -1,114 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link DoubleCoder} encodes {@link Double} values in 8 bytes using Java serialization.
- */
-public class DoubleCoder extends AtomicCoder<Double> {
-
-  @JsonCreator
-  public static DoubleCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final DoubleCoder INSTANCE = new DoubleCoder();
-
-  private DoubleCoder() {}
-
-  @Override
-  public void encode(Double value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Double");
-    }
-    new DataOutputStream(outStream).writeDouble(value);
-  }
-
-  @Override
-  public Double decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      return new DataInputStream(inStream).readDouble();
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always.
-   *         Floating-point operations are not guaranteed to be deterministic, even
-   *         if the storage format might be, so floating point representations are not
-   *         recommended for use in operations that require deterministic inputs.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Floating point encodings are not guaranteed to be deterministic.");
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link DoubleCoder#getEncodedElementByteSize} returns a constant.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Double value, Context context) {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code 8}, the byte size of a {@link Double} encoded using Java serialization.
-   */
-  @Override
-  protected long getEncodedElementByteSize(Double value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Double");
-    }
-    return 8;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DurationCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DurationCoder.java
deleted file mode 100644
index 46fbdde..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/DurationCoder.java
+++ /dev/null
@@ -1,98 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import org.joda.time.Duration;
-import org.joda.time.ReadableDuration;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} that encodes a joda {@link Duration} as a {@link Long} using the format of
- * {@link VarLongCoder}.
- */
-public class DurationCoder extends AtomicCoder<ReadableDuration> {
-
-  @JsonCreator
-  public static DurationCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final DurationCoder INSTANCE = new DurationCoder();
-
-  private final VarLongCoder longCoder = VarLongCoder.of();
-
-  private DurationCoder() {}
-
-  private Long toLong(ReadableDuration value) {
-    return value.getMillis();
-  }
-
-  private ReadableDuration fromLong(Long decoded) {
-    return Duration.millis(decoded);
-  }
-
-  @Override
-  public void encode(ReadableDuration value, OutputStream outStream, Context context)
-      throws CoderException, IOException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null ReadableDuration");
-    }
-    longCoder.encode(toLong(value), outStream, context);
-  }
-
-  @Override
-  public ReadableDuration decode(InputStream inStream, Context context)
-      throws CoderException, IOException {
-      return fromLong(longCoder.decode(inStream, context));
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}, because it is cheap to ascertain the byte size of a long.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(ReadableDuration value, Context context) {
-    return longCoder.isRegisterByteSizeObserverCheap(toLong(value), context);
-  }
-
-  @Override
-  public void registerByteSizeObserver(
-      ReadableDuration value, ElementByteSizeObserver observer, Context context) throws Exception {
-    longCoder.registerByteSizeObserver(toLong(value), observer, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java
deleted file mode 100644
index ffd516a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java
+++ /dev/null
@@ -1,87 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.api.services.datastore.DatastoreV1.Entity;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} for {@link Entity} objects based on their encoded Protocol Buffer form.
- */
-public class EntityCoder extends AtomicCoder<Entity> {
-
-  @JsonCreator
-  public static EntityCoder of() {
-    return INSTANCE;
-  }
-
-  /***************************/
-
-  private static final EntityCoder INSTANCE = new EntityCoder();
-
-  private EntityCoder() {}
-
-  @Override
-  public void encode(Entity value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Entity");
-    }
-
-    // Since Entity implements com.google.protobuf.MessageLite,
-    // we could directly use writeTo to write to a OutputStream object
-    outStream.write(java.nio.ByteBuffer.allocate(4).putInt(value.getSerializedSize()).array());
-    value.writeTo(outStream);
-    outStream.flush();
-  }
-
-  @Override
-  public Entity decode(InputStream inStream, Context context)
-      throws IOException {
-    byte[] entitySize = new byte[4];
-    inStream.read(entitySize, 0, 4);
-    int size = java.nio.ByteBuffer.wrap(entitySize).getInt();
-    byte[] data = new byte[size];
-    inStream.read(data, 0, size);
-    return Entity.parseFrom(data);
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(Entity value, Context context)
-      throws Exception {
-    return value.getSerializedSize();
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always.
-   *         A datastore kind can hold arbitrary {@link Object} instances, which
-   *         makes the encoding non-deterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Datastore encodings can hold arbitrary Object instances");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java
deleted file mode 100644
index 65c517f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java
+++ /dev/null
@@ -1,114 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.common.base.Converter;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} for joda {@link Instant} that encodes it as a big endian {@link Long}
- * shifted such that lexicographic ordering of the bytes corresponds to chronological order.
- */
-public class InstantCoder extends AtomicCoder<Instant> {
-
-  @JsonCreator
-  public static InstantCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final InstantCoder INSTANCE = new InstantCoder();
-
-  private final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
-
-  private InstantCoder() {}
-
-  /**
-   * Converts {@link Instant} to a {@code Long} representing its millis-since-epoch,
-   * but shifted so that the byte representation of negative values are lexicographically
-   * ordered before the byte representation of positive values.
-   *
-   * <p>This deliberately utilizes the well-defined overflow for {@code Long} values.
-   * See http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2
-   */
-  private static final Converter<Instant, Long> ORDER_PRESERVING_CONVERTER =
-      new Converter<Instant, Long>() {
-
-        @Override
-        protected Long doForward(Instant instant) {
-          return instant.getMillis() - Long.MIN_VALUE;
-        }
-
-        @Override
-        protected Instant doBackward(Long shiftedMillis) {
-          return new Instant(shiftedMillis + Long.MIN_VALUE);
-        }
-  };
-
-  @Override
-  public void encode(Instant value, OutputStream outStream, Context context)
-      throws CoderException, IOException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Instant");
-    }
-    longCoder.encode(ORDER_PRESERVING_CONVERTER.convert(value), outStream, context);
-  }
-
-  @Override
-  public Instant decode(InputStream inStream, Context context)
-      throws CoderException, IOException {
-    return ORDER_PRESERVING_CONVERTER.reverse().convert(longCoder.decode(inStream, context));
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. The byte size for a big endian long is a constant.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Instant value, Context context) {
-    return longCoder.isRegisterByteSizeObserverCheap(
-        ORDER_PRESERVING_CONVERTER.convert(value), context);
-  }
-
-  @Override
-  public void registerByteSizeObserver(
-      Instant value, ElementByteSizeObserver observer, Context context) throws Exception {
-    longCoder.registerByteSizeObserver(
-        ORDER_PRESERVING_CONVERTER.convert(value), observer, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java
deleted file mode 100644
index 55843ee..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java
+++ /dev/null
@@ -1,79 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.List;
-
-/**
- * An {@link IterableCoder} encodes any {@link Iterable} in the format
- * of {@link IterableLikeCoder}.
- *
- * @param <T> the type of the elements of the iterables being transcoded
- */
-public class IterableCoder<T> extends IterableLikeCoder<T, Iterable<T>> {
-
-  public static <T> IterableCoder<T> of(Coder<T> elemCoder) {
-    return new IterableCoder<>(elemCoder);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Internal operations below here.
-
-  @Override
-  protected final Iterable<T> decodeToIterable(List<T> decodedElements) {
-    return decodedElements;
-  }
-
-  @JsonCreator
-  public static IterableCoder<?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
-    return of(components.get(0));
-  }
-
-  /**
-   * Returns the first element in this iterable if it is non-empty,
-   * otherwise returns {@code null}.
-   */
-  public static <T> List<Object> getInstanceComponents(
-      Iterable<T> exampleValue) {
-    return getInstanceComponentsHelper(exampleValue);
-  }
-
-  protected IterableCoder(Coder<T> elemCoder) {
-    super(elemCoder, "Iterable");
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    addBoolean(result, PropertyNames.IS_STREAM_LIKE, true);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java
deleted file mode 100644
index 194e9ef..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java
+++ /dev/null
@@ -1,279 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.BufferedElementCountingOutputStream;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterable;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.common.base.Preconditions;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Observable;
-import java.util.Observer;
-
-/**
- * An abstract base class with functionality for assembling a
- * {@link Coder} for a class that implements {@code Iterable}.
- *
- * <p>To complete a subclass, implement the {@link #decodeToIterable} method. This superclass
- * will decode the elements in the input stream into a {@link List} and then pass them to that
- * method to be converted into the appropriate iterable type. Note that this means the input
- * iterables must fit into memory.
- *
- * <p>The format of this coder is as follows:
- *
- * <ul>
- *   <li>If the input {@link Iterable} has a known and finite size, then the size is written to the
- *       output stream in big endian format, followed by all of the encoded elements.</li>
- *   <li>If the input {@link Iterable} is not known to have a finite size, then each element
- *       of the input is preceded by {@code true} encoded as a byte (indicating "more data")
- *       followed by the encoded element, and terminated by {@code false} encoded as a byte.</li>
- * </ul>
- *
- * @param <T> the type of the elements of the {@code Iterable}s being transcoded
- * @param <IterableT> the type of the Iterables being transcoded
- */
-public abstract class IterableLikeCoder<T, IterableT extends Iterable<T>>
-    extends StandardCoder<IterableT> {
-  public Coder<T> getElemCoder() {
-    return elementCoder;
-  }
-
-  /**
-   * Builds an instance of {@code IterableT}, this coder's associated {@link Iterable}-like
-   * subtype, from a list of decoded elements.
-   */
-  protected abstract IterableT decodeToIterable(List<T> decodedElements);
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Internal operations below here.
-
-  private final Coder<T> elementCoder;
-  private final String iterableName;
-
-  /**
-   * Returns the first element in the iterable-like {@code exampleValue} if it is non-empty,
-   * otherwise returns {@code null}.
-   */
-  protected static <T, IterableT extends Iterable<T>>
-      List<Object> getInstanceComponentsHelper(IterableT exampleValue) {
-    for (T value : exampleValue) {
-      return Arrays.<Object>asList(value);
-    }
-    return null;
-  }
-
-  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");
-    this.elementCoder = elementCoder;
-    this.iterableName = iterableName;
-  }
-
-  @Override
-  public void encode(
-      IterableT iterable, OutputStream outStream, Context context)
-      throws IOException, CoderException  {
-    if (iterable == null) {
-      throw new CoderException("cannot encode a null " + iterableName);
-    }
-    Context nestedContext = context.nested();
-    DataOutputStream dataOutStream = new DataOutputStream(outStream);
-    if (iterable instanceof Collection) {
-      // We can know the size of the Iterable.  Use an encoding with a
-      // leading size field, followed by that many elements.
-      Collection<T> collection = (Collection<T>) iterable;
-      dataOutStream.writeInt(collection.size());
-      for (T elem : collection) {
-        elementCoder.encode(elem, dataOutStream, nestedContext);
-      }
-    } else {
-      // We don't know the size without traversing it so use a fixed size buffer
-      // and encode as many elements as possible into it before outputting the size followed
-      // by the elements.
-      dataOutStream.writeInt(-1);
-      BufferedElementCountingOutputStream countingOutputStream =
-          new BufferedElementCountingOutputStream(dataOutStream);
-      for (T elem : iterable) {
-        countingOutputStream.markElementStart();
-        elementCoder.encode(elem, countingOutputStream, nestedContext);
-      }
-      countingOutputStream.finish();
-    }
-    // Make sure all our output gets pushed to the underlying outStream.
-    dataOutStream.flush();
-  }
-
-  @Override
-  public IterableT decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    Context nestedContext = context.nested();
-    DataInputStream dataInStream = new DataInputStream(inStream);
-    int size = dataInStream.readInt();
-    if (size >= 0) {
-      List<T> elements = new ArrayList<>(size);
-      for (int i = 0; i < size; i++) {
-        elements.add(elementCoder.decode(dataInStream, nestedContext));
-      }
-      return decodeToIterable(elements);
-    } else {
-      List<T> elements = new ArrayList<>();
-      long count;
-      // We don't know the size a priori.  Check if we're done with
-      // each block of elements.
-      while ((count = VarInt.decodeLong(dataInStream)) > 0) {
-        while (count > 0) {
-          elements.add(elementCoder.decode(dataInStream, nestedContext));
-          count -= 1;
-        }
-      }
-      return decodeToIterable(elements);
-    }
-  }
-
-  @Override
-  public List<? extends Coder<?>> getCoderArguments() {
-    return Arrays.asList(elementCoder);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always.
-   * Encoding is not deterministic for the general {@link Iterable} case, as it depends
-   * upon the type of iterable. This may allow two objects to compare as equal
-   * while the encoding differs.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "IterableLikeCoder can not guarantee deterministic ordering.");
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true} if the iterable is of a known class that supports lazy counting
-   * of byte size, since that requires minimal extra computation.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(
-      IterableT iterable, Context context) {
-    return iterable instanceof ElementByteSizeObservableIterable;
-  }
-
-  @Override
-  public void registerByteSizeObserver(
-      IterableT iterable, ElementByteSizeObserver observer, Context context)
-      throws Exception {
-    if (iterable == null) {
-      throw new CoderException("cannot encode a null Iterable");
-    }
-    Context nestedContext = context.nested();
-
-    if (iterable instanceof ElementByteSizeObservableIterable) {
-      observer.setLazy();
-      ElementByteSizeObservableIterable<?, ?> observableIterable =
-          (ElementByteSizeObservableIterable<?, ?>) iterable;
-      observableIterable.addObserver(
-          new IteratorObserver(observer, iterable instanceof Collection));
-    } else {
-      if (iterable instanceof Collection) {
-        // We can know the size of the Iterable.  Use an encoding with a
-        // leading size field, followed by that many elements.
-        Collection<T> collection = (Collection<T>) iterable;
-        observer.update(4L);
-        for (T elem : collection) {
-          elementCoder.registerByteSizeObserver(elem, observer, nestedContext);
-        }
-      } else {
-        // TODO: Update to use an accurate count depending on size and count, currently we
-        // are under estimating the size by up to 10 bytes per block of data since we are
-        // not encoding the count prefix which occurs at most once per 64k of data and is upto
-        // 10 bytes long. Since we include the total count we can upper bound the underestimate
-        // to be 10 / 65536 ~= 0.0153% of the actual size.
-        observer.update(4L);
-        long count = 0;
-        for (T elem : iterable) {
-          count += 1;
-          elementCoder.registerByteSizeObserver(elem, observer, nestedContext);
-        }
-        if (count > 0) {
-          // Update the length based upon the number of counted elements, this helps
-          // eliminate the case where all the elements are encoded in the first block and
-          // it is quite short (e.g. Long.MAX_VALUE nulls encoded with VoidCoder).
-          observer.update(VarInt.getLength(count));
-        }
-        // Update with the terminator byte.
-        observer.update(1L);
-      }
-    }
-  }
-
-  /**
-   * An observer that gets notified when an observable iterator
-   * returns a new value. This observer just notifies an outerObserver
-   * about this event. Additionally, the outerObserver is notified
-   * about additional separators that are transparently added by this
-   * coder.
-   */
-  private class IteratorObserver implements Observer {
-    private final ElementByteSizeObserver outerObserver;
-    private final boolean countable;
-
-    public IteratorObserver(ElementByteSizeObserver outerObserver,
-                            boolean countable) {
-      this.outerObserver = outerObserver;
-      this.countable = countable;
-
-      if (countable) {
-        // Additional 4 bytes are due to size.
-        outerObserver.update(4L);
-      } else {
-        // Additional 5 bytes are due to size = -1 (4 bytes) and
-        // hasNext = false (1 byte).
-        outerObserver.update(5L);
-      }
-    }
-
-    @Override
-    public void update(Observable obs, Object obj) {
-      if (!(obj instanceof Long)) {
-        throw new AssertionError("unexpected parameter object");
-      }
-
-      if (countable) {
-        outerObserver.update(obs, obj);
-      } else {
-        // Additional 1 byte is due to hasNext = true flag.
-        outerObserver.update(obs, 1 + (long) obj);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java
deleted file mode 100644
index 6e2833e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java
+++ /dev/null
@@ -1,170 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.common.io.ByteStreams;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.FilterInputStream;
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-
-/**
- * A coder for JAXB annotated objects. This coder uses JAXB marshalling/unmarshalling mechanisms
- * to encode/decode the objects. Users must provide the {@code Class} of the JAXB annotated object.
- *
- * @param <T> type of JAXB annotated objects that will be serialized.
- */
-public class JAXBCoder<T> extends AtomicCoder<T> {
-
-  private final Class<T> jaxbClass;
-  private transient Marshaller jaxbMarshaller = null;
-  private transient Unmarshaller jaxbUnmarshaller = null;
-
-  public Class<T> getJAXBClass() {
-    return jaxbClass;
-  }
-
-  private JAXBCoder(Class<T> jaxbClass) {
-    this.jaxbClass = jaxbClass;
-  }
-
-  /**
-   * Create a coder for a given type of JAXB annotated objects.
-   *
-   * @param jaxbClass the {@code Class} of the JAXB annotated objects.
-   */
-  public static <T> JAXBCoder<T> of(Class<T> jaxbClass) {
-    return new JAXBCoder<>(jaxbClass);
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context)
-      throws CoderException, IOException {
-    try {
-      if (jaxbMarshaller == null) {
-        JAXBContext jaxbContext = JAXBContext.newInstance(jaxbClass);
-        jaxbMarshaller = jaxbContext.createMarshaller();
-      }
-      if (!context.isWholeStream) {
-        try {
-          long size = getEncodedElementByteSize(value, Context.OUTER);
-          // record the number of bytes the XML consists of so when reading we only read the encoded
-          // value
-          VarInt.encode(size, outStream);
-        } catch (Exception e) {
-          throw new CoderException(
-              "An Exception occured while trying to get the size of an encoded representation", e);
-        }
-      }
-
-      jaxbMarshaller.marshal(value, new CloseIgnoringOutputStream(outStream));
-    } catch (JAXBException e) {
-      throw new CoderException(e);
-    }
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws CoderException, IOException {
-    try {
-      if (jaxbUnmarshaller == null) {
-        JAXBContext jaxbContext = JAXBContext.newInstance(jaxbClass);
-        jaxbUnmarshaller = jaxbContext.createUnmarshaller();
-      }
-
-      InputStream stream = inStream;
-      if (!context.isWholeStream) {
-        long limit = VarInt.decodeLong(inStream);
-        stream = ByteStreams.limit(inStream, limit);
-      }
-      @SuppressWarnings("unchecked")
-      T obj = (T) jaxbUnmarshaller.unmarshal(new CloseIgnoringInputStream(stream));
-      return obj;
-    } catch (JAXBException e) {
-      throw new CoderException(e);
-    }
-  }
-
-  @Override
-  public String getEncodingId() {
-    return getJAXBClass().getName();
-  }
-
-  private static class CloseIgnoringInputStream extends FilterInputStream {
-
-    protected CloseIgnoringInputStream(InputStream in) {
-      super(in);
-    }
-
-    @Override
-    public void close() {
-      // Do nothing. JAXB closes the underlying stream so we must filter out those calls.
-    }
-  }
-
-  private static class CloseIgnoringOutputStream extends FilterOutputStream {
-
-    protected CloseIgnoringOutputStream(OutputStream out) {
-      super(out);
-    }
-
-    @Override
-    public void close() throws IOException {
-      // JAXB closes the underlying stream so we must filter out those calls.
-    }
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////
-  // JSON Serialization details below
-
-  private static final String JAXB_CLASS = "jaxb_class";
-
-  /**
-   * Constructor for JSON deserialization only.
-   */
-  @JsonCreator
-  public static <T> JAXBCoder<T> of(
-      @JsonProperty(JAXB_CLASS) String jaxbClassName) {
-    try {
-      @SuppressWarnings("unchecked")
-      Class<T> jaxbClass = (Class<T>) Class.forName(jaxbClassName);
-      return of(jaxbClass);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    Structs.addString(result, JAXB_CLASS, jaxbClass.getName());
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java
deleted file mode 100644
index 227c546..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java
+++ /dev/null
@@ -1,163 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * A {@code KvCoder} encodes {@link KV}s.
- *
- * @param <K> the type of the keys of the KVs being transcoded
- * @param <V> the type of the values of the KVs being transcoded
- */
-public class KvCoder<K, V> extends KvCoderBase<KV<K, V>> {
-  public static <K, V> KvCoder<K, V> of(Coder<K> keyCoder,
-                                        Coder<V> valueCoder) {
-    return new KvCoder<>(keyCoder, valueCoder);
-  }
-
-  @JsonCreator
-  public static KvCoder<?, ?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 2,
-        "Expecting 2 components, got " + components.size());
-    return of(components.get(0), components.get(1));
-  }
-
-  public static <K, V> List<Object> getInstanceComponents(
-      KV<K, V> exampleValue) {
-    return Arrays.asList(
-        exampleValue.getKey(),
-        exampleValue.getValue());
-  }
-
-  public Coder<K> getKeyCoder() {
-    return keyCoder;
-  }
-
-  public Coder<V> getValueCoder() {
-    return valueCoder;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private final Coder<K> keyCoder;
-  private final Coder<V> valueCoder;
-
-  private KvCoder(Coder<K> keyCoder, Coder<V> valueCoder) {
-    this.keyCoder = keyCoder;
-    this.valueCoder = valueCoder;
-  }
-
-  @Override
-  public void encode(KV<K, V> kv, OutputStream outStream, Context context)
-      throws IOException, CoderException  {
-    if (kv == null) {
-      throw new CoderException("cannot encode a null KV");
-    }
-    Context nestedContext = context.nested();
-    keyCoder.encode(kv.getKey(), outStream, nestedContext);
-    valueCoder.encode(kv.getValue(), outStream, nestedContext);
-  }
-
-  @Override
-  public KV<K, V> decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    Context nestedContext = context.nested();
-    K key = keyCoder.decode(inStream, nestedContext);
-    V value = valueCoder.decode(inStream, nestedContext);
-    return KV.of(key, value);
-  }
-
-  @Override
-  public List<? extends Coder<?>> getCoderArguments() {
-    return Arrays.asList(keyCoder, valueCoder);
-  }
-
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    verifyDeterministic("Key coder must be deterministic", getKeyCoder());
-    verifyDeterministic("Value coder must be deterministic", getValueCoder());
-  }
-
-  @Override
-  public boolean consistentWithEquals() {
-    return keyCoder.consistentWithEquals() && valueCoder.consistentWithEquals();
-  }
-
-  @Override
-  public Object structuralValue(KV<K, V> kv) throws Exception {
-    if (consistentWithEquals()) {
-      return kv;
-    } else {
-      return KV.of(getKeyCoder().structuralValue(kv.getKey()),
-                   getValueCoder().structuralValue(kv.getValue()));
-    }
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    addBoolean(result, PropertyNames.IS_PAIR_LIKE, true);
-    return result;
-  }
-
-  /**
-   * Returns whether both keyCoder and valueCoder are considered not expensive.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(KV<K, V> kv, Context context) {
-    return keyCoder.isRegisterByteSizeObserverCheap(kv.getKey(),
-                                                    context.nested())
-        && valueCoder.isRegisterByteSizeObserverCheap(kv.getValue(),
-                                                      context.nested());
-  }
-
-  /**
-   * Notifies ElementByteSizeObserver about the byte size of the
-   * encoded value using this coder.
-   */
-  @Override
-  public void registerByteSizeObserver(
-      KV<K, V> kv, ElementByteSizeObserver observer, Context context)
-      throws Exception {
-    if (kv == null) {
-      throw new CoderException("cannot encode a null KV");
-    }
-    keyCoder.registerByteSizeObserver(
-        kv.getKey(), observer, context.nested());
-    valueCoder.registerByteSizeObserver(
-        kv.getValue(), observer, context.nested());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java
deleted file mode 100644
index 361ffb9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java
+++ /dev/null
@@ -1,62 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.List;
-
-/**
- * A abstract base class for KvCoder. Works around a Jackson2 bug tickled when building
- * {@link KvCoder} directly (as of this writing, Jackson2 walks off the end of
- * an array when it tries to deserialize a class with multiple generic type
- * parameters).  This class should be removed when possible.
- *
- * @param <T> the type of values being transcoded
- */
-@Deprecated
-public abstract class KvCoderBase<T> extends StandardCoder<T> {
-  /**
-   * A constructor used only for decoding from JSON.
-   *
-   * @param typeId present in the JSON encoding, but unused
-   * @param isPairLike present in the JSON encoding, but unused
-   */
-  @Deprecated
-  @JsonCreator
-  public static KvCoderBase<?> of(
-      // N.B. typeId is a required parameter here, since a field named "@type"
-      // is presented to the deserializer as an input.
-      //
-      // If this method did not consume the field, Jackson2 would observe an
-      // unconsumed field and a returned value of a derived type.  So Jackson2
-      // would attempt to update the returned value with the unconsumed field
-      // data.  The standard JsonDeserializer does not implement a mechanism for
-      // updating constructed values, so it would throw an exception, causing
-      // deserialization to fail.
-      @JsonProperty(value = "@type", required = false) String typeId,
-      @JsonProperty(value = PropertyNames.IS_PAIR_LIKE, required = false) boolean isPairLike,
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
-    return KvCoder.of(components);
-  }
-
-  protected KvCoderBase() {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java
deleted file mode 100644
index cfb3ab7..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.List;
-
-/**
- * A {@link Coder} for {@link List}, using the format of {@link IterableLikeCoder}.
- *
- * @param <T> the type of the elements of the Lists being transcoded
- */
-public class ListCoder<T> extends IterableLikeCoder<T, List<T>> {
-
-  public static <T> ListCoder<T> of(Coder<T> elemCoder) {
-    return new ListCoder<>(elemCoder);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Internal operations below here.
-
-  @Override
-  protected final List<T> decodeToIterable(List<T> decodedElements) {
-    return decodedElements;
-  }
-
-  @JsonCreator
-  public static ListCoder<?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
-    return of((Coder<?>) components.get(0));
-  }
-
-  /**
-   * Returns the first element in this list if it is non-empty,
-   * otherwise returns {@code null}.
-   */
-  public static <T> List<Object> getInstanceComponents(List<T> exampleValue) {
-    return getInstanceComponentsHelper(exampleValue);
-  }
-
-  protected ListCoder(Coder<T> elemCoder) {
-    super(elemCoder, "List");
-  }
-
-  /**
-   * List sizes are always known, so ListIterable may be deterministic while
-   * the general IterableLikeCoder is not.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    verifyDeterministic(
-        "ListCoder.elemCoder must be deterministic", getElemCoder());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java
deleted file mode 100644
index 5a60ab5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java
+++ /dev/null
@@ -1,161 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-/**
- * A {@link Coder} for {@link Map Maps} that encodes them according to provided
- * coders for keys and values.
- *
- * @param <K> the type of the keys of the KVs being transcoded
- * @param <V> the type of the values of the KVs being transcoded
- */
-public class MapCoder<K, V> extends MapCoderBase<Map<K, V>> {
-  /**
-   * Produces a MapCoder with the given keyCoder and valueCoder.
-   */
-  public static <K, V> MapCoder<K, V> of(
-      Coder<K> keyCoder,
-      Coder<V> valueCoder) {
-    return new MapCoder<>(keyCoder, valueCoder);
-  }
-
-  @JsonCreator
-  public static MapCoder<?, ?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 2,
-        "Expecting 2 components, got " + components.size());
-    return of((Coder<?>) components.get(0), (Coder<?>) components.get(1));
-  }
-
-  /**
-   * Returns the key and value for an arbitrary element of this map,
-   * if it is non-empty, otherwise returns {@code null}.
-   */
-   public static <K, V> List<Object> getInstanceComponents(
-       Map<K, V> exampleValue) {
-     for (Map.Entry<K, V> entry : exampleValue.entrySet()) {
-       return Arrays.asList(entry.getKey(), entry.getValue());
-     }
-     return null;
-   }
-
-  public Coder<K> getKeyCoder() {
-    return keyCoder;
-  }
-
-  public Coder<V> getValueCoder() {
-    return valueCoder;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  Coder<K> keyCoder;
-  Coder<V> valueCoder;
-
-  MapCoder(Coder<K> keyCoder, Coder<V> valueCoder) {
-    this.keyCoder = keyCoder;
-    this.valueCoder = valueCoder;
-  }
-
-  @Override
-  public void encode(
-      Map<K, V> map,
-      OutputStream outStream,
-      Context context)
-      throws IOException, CoderException  {
-    if (map == null) {
-      throw new CoderException("cannot encode a null Map");
-    }
-    DataOutputStream dataOutStream = new DataOutputStream(outStream);
-    dataOutStream.writeInt(map.size());
-    for (Entry<K, V> entry : map.entrySet()) {
-      keyCoder.encode(entry.getKey(), outStream, context.nested());
-      valueCoder.encode(entry.getValue(), outStream, context.nested());
-    }
-    dataOutStream.flush();
-  }
-
-  @Override
-  public Map<K, V> decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    DataInputStream dataInStream = new DataInputStream(inStream);
-    int size = dataInStream.readInt();
-    Map<K, V> retval = Maps.newHashMapWithExpectedSize(size);
-    for (int i = 0; i < size; ++i) {
-      K key = keyCoder.decode(inStream, context.nested());
-      V value = valueCoder.decode(inStream, context.nested());
-      retval.put(key, value);
-    }
-    return retval;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return a {@link List} containing the key coder at index 0 at the and value coder at index 1.
-   */
-  @Override
-  public List<? extends Coder<?>> getCoderArguments() {
-    return Arrays.asList(keyCoder, valueCoder);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always. Not all maps have a deterministic encoding.
-   * For example, {@code HashMap} comparison does not depend on element order, so
-   * two {@code HashMap} instances may be equal but produce different encodings.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Ordering of entries in a Map may be non-deterministic.");
-  }
-
-  @Override
-  public void registerByteSizeObserver(
-      Map<K, V> map, ElementByteSizeObserver observer, Context context)
-      throws Exception {
-    observer.update(4L);
-    for (Entry<K, V> entry : map.entrySet()) {
-      keyCoder.registerByteSizeObserver(
-          entry.getKey(), observer, context.nested());
-      valueCoder.registerByteSizeObserver(
-          entry.getValue(), observer, context.nested());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java
deleted file mode 100644
index 1fc3f57..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java
+++ /dev/null
@@ -1,55 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.List;
-
-/**
- * A abstract base class for MapCoder. Works around a Jackson2 bug tickled when building
- * {@link MapCoder} directly (as of this writing, Jackson2 walks off the end of
- * an array when it tries to deserialize a class with multiple generic type
- * parameters).  This should be removed in favor of a better workaround.
- * @param <T> the type of values being transcoded
- */
-@Deprecated
-public abstract class MapCoderBase<T> extends StandardCoder<T> {
-  @Deprecated
-  @JsonCreator
-  public static MapCoderBase<?> of(
-      // N.B. typeId is a required parameter here, since a field named "@type"
-      // is presented to the deserializer as an input.
-      //
-      // If this method did not consume the field, Jackson2 would observe an
-      // unconsumed field and a returned value of a derived type.  So Jackson2
-      // would attempt to update the returned value with the unconsumed field
-      // data, The standard JsonDeserializer does not implement a mechanism for
-      // updating constructed values, so it would throw an exception, causing
-      // deserialization to fail.
-      @JsonProperty(value = "@type", required = false) String typeId,
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    return MapCoder.of(components);
-  }
-
-  protected MapCoderBase() {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/NullableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/NullableCoder.java
deleted file mode 100644
index fc12a03..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/NullableCoder.java
+++ /dev/null
@@ -1,179 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.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;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link NullableCoder} encodes nullable values of type {@code T} using a nested
- * {@code Coder<T>} that does not tolerate {@code null} values. {@link NullableCoder} uses
- * exactly 1 byte per entry to indicate whether the value is {@code null}, then adds the encoding
- * of the inner coder for non-null values.
- *
- * @param <T> the type of the values being transcoded
- */
-public class NullableCoder<T> extends StandardCoder<T> {
-  public static <T> NullableCoder<T> of(Coder<T> valueCoder) {
-    if (valueCoder instanceof NullableCoder) {
-      return (NullableCoder<T>) valueCoder;
-    }
-    return new NullableCoder<>(valueCoder);
-  }
-
-  @JsonCreator
-  public static NullableCoder<?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 components, got " + components.size());
-    return of(components.get(0));
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private final Coder<T> valueCoder;
-  private static final int ENCODE_NULL = 0;
-  private static final int ENCODE_PRESENT = 1;
-
-  private NullableCoder(Coder<T> valueCoder) {
-    this.valueCoder = valueCoder;
-  }
-
-  @Override
-  public void encode(@Nullable T value, OutputStream outStream, Context context)
-      throws IOException, CoderException  {
-    if (value == null) {
-      outStream.write(ENCODE_NULL);
-    } else {
-      outStream.write(ENCODE_PRESENT);
-      valueCoder.encode(value, outStream, context.nested());
-    }
-  }
-
-  @Override
-  @Nullable
-  public T decode(InputStream inStream, Context context) throws IOException, CoderException {
-    int b = inStream.read();
-    if (b == ENCODE_NULL) {
-      return null;
-    } else if (b != ENCODE_PRESENT) {
-        throw new CoderException(String.format(
-            "NullableCoder expects either a byte valued %s (null) or %s (present), got %s",
-            ENCODE_NULL, ENCODE_PRESENT, b));
-    }
-    return valueCoder.decode(inStream, context.nested());
-  }
-
-  @Override
-  public List<Coder<T>> getCoderArguments() {
-    return ImmutableList.of(valueCoder);
-  }
-
-  /**
-   * {@code NullableCoder} is deterministic if the nested {@code Coder} is.
-   *
-   * {@inheritDoc}
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    verifyDeterministic("Value coder must be deterministic", valueCoder);
-  }
-
-  /**
-   * {@code NullableCoder} is consistent with equals if the nested {@code Coder} is.
-   *
-   * {@inheritDoc}
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return valueCoder.consistentWithEquals();
-  }
-
-  @Override
-  public Object structuralValue(@Nullable T value) throws Exception {
-    if (value == null) {
-      return Optional.absent();
-    }
-    return Optional.of(valueCoder.structuralValue(value));
-  }
-
-  /**
-   * Overridden to short-circuit the default {@code StandardCoder} behavior of encoding and
-   * counting the bytes. The size is known (1 byte) when {@code value} is {@code null}, otherwise
-   * the size is 1 byte plus the size of nested {@code Coder}'s encoding of {@code value}.
-   *
-   * {@inheritDoc}
-   */
-  @Override
-  public void registerByteSizeObserver(
-      @Nullable T value, ElementByteSizeObserver observer, Context context) throws Exception {
-    observer.update(1);
-    if (value != null) {
-      valueCoder.registerByteSizeObserver(value, observer, context.nested());
-    }
-  }
-
-  /**
-   * Overridden to short-circuit the default {@code StandardCoder} behavior of encoding and
-   * counting the bytes. The size is known (1 byte) when {@code value} is {@code null}, otherwise
-   * the size is 1 byte plus the size of nested {@code Coder}'s encoding of {@code value}.
-   *
-   * {@inheritDoc}
-   */
-  @Override
-  protected long getEncodedElementByteSize(@Nullable T value, Context context) throws Exception {
-    if (value == null) {
-      return 1;
-    }
-
-    if (valueCoder instanceof StandardCoder) {
-      // If valueCoder is a StandardCoder then we can ask it directly for the encoded size of
-      // the value, adding 1 byte to count the null indicator.
-      return 1  + ((StandardCoder<T>) valueCoder)
-          .getEncodedElementByteSize(value, context.nested());
-    }
-
-    // If value is not a StandardCoder then fall back to the default StandardCoder behavior
-    // of encoding and counting the bytes. The encoding will include the null indicator byte.
-    return super.getEncodedElementByteSize(value, context);
-  }
-
-  /**
-   * {@code NullableCoder} is cheap if {@code valueCoder} is cheap.
-   *
-   * {@inheritDoc}
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(@Nullable T value, Context context) {
-    return valueCoder.isRegisterByteSizeObserverCheap(value, context.nested());
-  }
-}


[02/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatchers.java
deleted file mode 100644
index 689c387..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatchers.java
+++ /dev/null
@@ -1,1181 +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 com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.base.MoreObjects;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * Static class for building and using {@link SerializableMatcher} instances.
- *
- * <p>Most matchers are wrappers for hamcrest's {@link Matchers}. Please be familiar with the
- * documentation there. Values retained by a {@link SerializableMatcher} are required to be
- * serializable, either via Java serialization or via a provided {@link Coder}.
- *
- * <p>The following matchers are novel to Dataflow:
- * <ul>
- * <li>{@link #kvWithKey} for matching just the key of a {@link KV}.
- * <li>{@link #kvWithValue} for matching just the value of a {@link KV}.
- * <li>{@link #kv} for matching the key and value of a {@link KV}.
- * </ul>
- *
- * <p>For example, to match a group from
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}, which has type
- * {@code KV<K, Iterable<V>>} for some {@code K} and {@code V} and where the order of the iterable
- * is undefined, use a matcher like
- * {@code kv(equalTo("some key"), containsInAnyOrder(1, 2, 3))}.
- */
-class SerializableMatchers implements Serializable {
-
-  // Serializable only because of capture by anonymous inner classes
-  private SerializableMatchers() { } // not instantiable
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#allOf(Iterable)}.
-   */
-  public static <T> SerializableMatcher<T>
-  allOf(Iterable<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final Iterable<Matcher<? super T>> matchers = (Iterable) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.allOf(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#allOf(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T> allOf(final SerializableMatcher<T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.allOf(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#anyOf(Iterable)}.
-   */
-  public static <T> SerializableMatcher<T>
-  anyOf(Iterable<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final Iterable<Matcher<? super T>> matchers = (Iterable) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.anyOf(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#anyOf(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T> anyOf(final SerializableMatcher<T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.anyOf(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#anything()}.
-   */
-  public static SerializableMatcher<Object> anything() {
-    return fromSupplier(new SerializableSupplier<Matcher<Object>>() {
-      @Override
-      public Matcher<Object> get() {
-        return Matchers.anything();
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContaining(Object[])}.
-   */
-  @SafeVarargs
-  public static <T extends Serializable> SerializableMatcher<T[]>
-  arrayContaining(final T... items) {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContaining(items);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContaining(Object[])}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are
-   * explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T[]> arrayContaining(Coder<T> coder, T... items) {
-
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContaining(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContaining(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T[]>
-  arrayContaining(final SerializableMatcher<? super T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.<T>arrayContaining(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContaining(List)}.
-   */
-  public static <T> SerializableMatcher<T[]>
-  arrayContaining(List<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final List<Matcher<? super T>> matchers = (List) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContaining(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContainingInAnyOrder(Object[])}.
-   */
-  @SafeVarargs
-  public static <T extends Serializable> SerializableMatcher<T[]>
-  arrayContainingInAnyOrder(final T... items) {
-
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContainingInAnyOrder(items);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContainingInAnyOrder(Object[])}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are
-   * explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T[]> arrayContainingInAnyOrder(Coder<T> coder, T... items) {
-
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContaining(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContainingInAnyOrder(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T[]> arrayContainingInAnyOrder(
-      final SerializableMatcher<? super T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.<T>arrayContainingInAnyOrder(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayContainingInAnyOrder(Collection)}.
-   */
-  public static <T> SerializableMatcher<T[]> arrayContainingInAnyOrder(
-      Collection<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final Collection<Matcher<? super T>> matchers = (Collection) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayContainingInAnyOrder(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayWithSize(int)}.
-   */
-  public static <T> SerializableMatcher<T[]> arrayWithSize(final int size) {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayWithSize(size);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#arrayWithSize(Matcher)}.
-   */
-  public static <T> SerializableMatcher<T[]> arrayWithSize(
-      final SerializableMatcher<? super Integer> sizeMatcher) {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.arrayWithSize(sizeMatcher);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#closeTo(double,double)}.
-   */
-  public static SerializableMatcher<Double> closeTo(final double target, final double error) {
-    return fromSupplier(new SerializableSupplier<Matcher<Double>>() {
-      @Override
-      public Matcher<Double> get() {
-        return Matchers.closeTo(target, error);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#contains(Object[])}.
-   */
-  @SafeVarargs
-  public static <T extends Serializable> SerializableMatcher<Iterable<? extends T>> contains(
-      final T... items) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.contains(items);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#contains(Object[])}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are
-   * explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<Iterable<? extends T>>
-  contains(Coder<T> coder, T... items) {
-
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.containsInAnyOrder(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#contains(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<Iterable<? extends T>> contains(
-      final SerializableMatcher<? super T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.<T>contains(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#contains(List)}.
-   */
-  public static <T extends Serializable> SerializableMatcher<Iterable<? extends T>> contains(
-      List<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final List<Matcher<? super T>> matchers = (List) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.contains(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#containsInAnyOrder(Object[])}.
-   */
-  @SafeVarargs
-  public static <T extends Serializable> SerializableMatcher<Iterable<? extends T>>
-  containsInAnyOrder(final T... items) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.containsInAnyOrder(items);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#containsInAnyOrder(Object[])}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<Iterable<? extends T>>
-  containsInAnyOrder(Coder<T> coder, T... items) {
-
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.containsInAnyOrder(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#containsInAnyOrder(Matcher[])}.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<Iterable<? extends T>> containsInAnyOrder(
-      final SerializableMatcher<? super T>... matchers) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.<T>containsInAnyOrder(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#containsInAnyOrder(Collection)}.
-   */
-  public static <T> SerializableMatcher<Iterable<? extends T>> containsInAnyOrder(
-      Collection<SerializableMatcher<? super T>> serializableMatchers) {
-
-    @SuppressWarnings({"rawtypes", "unchecked"}) // safe covariant cast
-    final Collection<Matcher<? super T>> matchers = (Collection) serializableMatchers;
-
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.containsInAnyOrder(matchers);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#containsString}.
-   */
-  public static SerializableMatcher<String> containsString(final String substring) {
-    return fromSupplier(new SerializableSupplier<Matcher<String>>() {
-      @Override
-      public Matcher<String> get() {
-        return Matchers.containsString(substring);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#empty()}.
-   */
-  public static <T> SerializableMatcher<Collection<? extends T>> empty() {
-    return fromSupplier(new SerializableSupplier<Matcher<Collection<? extends T>>>() {
-      @Override
-      public Matcher<Collection<? extends T>> get() {
-        return Matchers.empty();
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#emptyArray()}.
-   */
-  public static <T> SerializableMatcher<T[]> emptyArray() {
-    return fromSupplier(new SerializableSupplier<Matcher<T[]>>() {
-      @Override
-      public Matcher<T[]> get() {
-        return Matchers.emptyArray();
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#emptyIterable()}.
-   */
-  public static <T> SerializableMatcher<Iterable<? extends T>> emptyIterable() {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? extends T>>>() {
-      @Override
-      public Matcher<Iterable<? extends T>> get() {
-        return Matchers.emptyIterable();
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#endsWith}.
-   */
-  public static SerializableMatcher<String> endsWith(final String substring) {
-    return fromSupplier(new SerializableSupplier<Matcher<String>>() {
-      @Override
-      public Matcher<String> get() {
-        return Matchers.endsWith(substring);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#equalTo()}.
-   */
-  public static <T extends Serializable> SerializableMatcher<T> equalTo(final T expected) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.equalTo(expected);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#equalTo()}.
-   *
-   * <p>The expected value of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T> SerializableMatcher<T> equalTo(Coder<T> coder, T expected) {
-
-    final SerializableSupplier<T> expectedSupplier = new SerializableViaCoder<>(coder, expected);
-
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.equalTo(expectedSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#greaterThan()}.
-   */
-  public static <T extends Comparable<T> & Serializable> SerializableMatcher<T>
-  greaterThan(final T target) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.greaterThan(target);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#greaterThan()}.
-   *
-   * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T extends Comparable<T> & Serializable> SerializableMatcher<T>
-  greaterThan(final Coder<T> coder, T target) {
-    final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.greaterThan(targetSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#greaterThanOrEqualTo()}.
-   */
-  public static <T extends Comparable<T>> SerializableMatcher<T> greaterThanOrEqualTo(
-      final T target) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.greaterThanOrEqualTo(target);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#greaterThanOrEqualTo()}.
-   *
-   * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T extends Comparable<T> & Serializable> SerializableMatcher<T>
-  greaterThanOrEqualTo(final Coder<T> coder, T target) {
-    final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.greaterThanOrEqualTo(targetSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasItem(Object)}.
-   */
-  public static <T extends Serializable> SerializableMatcher<Iterable<? super T>> hasItem(
-      final T target) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? super T>>>() {
-      @Override
-      public Matcher<Iterable<? super T>> get() {
-        return Matchers.hasItem(target);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasItem(Object)}.
-   *
-   * <p>The item of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T> SerializableMatcher<Iterable<? super T>> hasItem(Coder<T> coder, T target) {
-    final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target);
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? super T>>>() {
-      @Override
-      public Matcher<Iterable<? super T>> get() {
-        return Matchers.hasItem(targetSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasItem(Matcher)}.
-   */
-  public static <T> SerializableMatcher<Iterable<? super T>> hasItem(
-      final SerializableMatcher<? super T> matcher) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<? super T>>>() {
-      @Override
-      public Matcher<Iterable<? super T>> get() {
-        return Matchers.hasItem(matcher);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasSize(int)}.
-   */
-  public static <T> SerializableMatcher<Collection<? extends T>> hasSize(final int size) {
-    return fromSupplier(new SerializableSupplier<Matcher<Collection<? extends T>>>() {
-      @Override
-      public Matcher<Collection<? extends T>> get() {
-        return Matchers.hasSize(size);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasSize(Matcher)}.
-   */
-  public static <T> SerializableMatcher<Collection<? extends T>> hasSize(
-      final SerializableMatcher<? super Integer> sizeMatcher) {
-    return fromSupplier(new SerializableSupplier<Matcher<Collection<? extends T>>>() {
-      @Override
-      public Matcher<Collection<? extends T>> get() {
-        return Matchers.hasSize(sizeMatcher);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#iterableWithSize(int)}.
-   */
-  public static <T> SerializableMatcher<Iterable<T>> iterableWithSize(final int size) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<T>>>() {
-      @Override
-      public Matcher<Iterable<T>> get() {
-        return Matchers.iterableWithSize(size);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#iterableWithSize(Matcher)}.
-   */
-  public static <T> SerializableMatcher<Iterable<T>> iterableWithSize(
-      final SerializableMatcher<? super Integer> sizeMatcher) {
-    return fromSupplier(new SerializableSupplier<Matcher<Iterable<T>>>() {
-      @Override
-      public Matcher<Iterable<T>> get() {
-        return Matchers.iterableWithSize(sizeMatcher);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Collection)}.
-   */
-  public static <T extends Serializable> SerializableMatcher<T>
-  isIn(final Collection<T> collection) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isIn(collection);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Collection)}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}.
-   * They are explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T> SerializableMatcher<T> isIn(Coder<T> coder, Collection<T> collection) {
-    @SuppressWarnings("unchecked")
-    T[] items = (T[]) collection.toArray();
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isIn(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Object[])}.
-   */
-  public static <T extends Serializable> SerializableMatcher<T> isIn(final T[] items) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isIn(items);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Object[])}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}.
-   * They are explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T> SerializableMatcher<T> isIn(Coder<T> coder, T[] items) {
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isIn(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isOneOf}.
-   */
-  @SafeVarargs
-  public static <T extends Serializable> SerializableMatcher<T> isOneOf(final T... elems) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isOneOf(elems);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isOneOf}.
-   *
-   * <p>The items of type {@code T} will be serialized using the provided {@link Coder}.
-   * They are explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  @SafeVarargs
-  public static <T> SerializableMatcher<T> isOneOf(Coder<T> coder, T... items) {
-    final SerializableSupplier<T[]> itemsSupplier =
-        new SerializableArrayViaCoder<>(coder, items);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.isOneOf(itemsSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with the specified key.
-   */
-  public static <K extends Serializable, V> SerializableMatcher<KV<? extends K, ? extends V>>
-  kvWithKey(K key) {
-    return new KvKeyMatcher<K, V>(equalTo(key));
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with the specified key.
-   *
-   * <p>The key of type {@code K} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <K, V> SerializableMatcher<KV<? extends K, ? extends V>>
-  kvWithKey(Coder<K> coder, K key) {
-    return new KvKeyMatcher<K, V>(equalTo(coder, key));
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with matching key.
-   */
-  public static <K, V> SerializableMatcher<KV<? extends K, ? extends V>> kvWithKey(
-      final SerializableMatcher<? super K> keyMatcher) {
-    return new KvKeyMatcher<K, V>(keyMatcher);
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with the specified value.
-   */
-  public static <K, V extends Serializable> SerializableMatcher<KV<? extends K, ? extends V>>
-  kvWithValue(V value) {
-    return new KvValueMatcher<K, V>(equalTo(value));
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with the specified value.
-   *
-   * <p>The value of type {@code V} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <K, V> SerializableMatcher<KV<? extends K, ? extends V>>
-  kvWithValue(Coder<V> coder, V value) {
-    return new KvValueMatcher<K, V>(equalTo(coder, value));
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with matching value.
-   */
-  public static <K, V> SerializableMatcher<KV<? extends K, ? extends V>> kvWithValue(
-      final SerializableMatcher<? super V> valueMatcher) {
-    return new KvValueMatcher<>(valueMatcher);
-  }
-
-  /**
-   * A {@link SerializableMatcher} that matches any {@link KV} with matching key and value.
-   */
-  public static <K, V> SerializableMatcher<KV<? extends K, ? extends V>> kv(
-      final SerializableMatcher<? super K> keyMatcher,
-      final SerializableMatcher<? super V> valueMatcher) {
-
-    return SerializableMatchers.<KV<? extends K, ? extends V>>allOf(
-        SerializableMatchers.<K, V>kvWithKey(keyMatcher),
-        SerializableMatchers.<K, V>kvWithValue(valueMatcher));
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#lessThan()}.
-   */
-  public static <T extends Comparable<T> & Serializable> SerializableMatcher<T> lessThan(
-      final T target) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.lessThan(target);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#lessThan()}.
-   *
-   * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T extends Comparable<T>> SerializableMatcher<T>
-  lessThan(Coder<T> coder, T target) {
-    final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.lessThan(targetSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#lessThanOrEqualTo()}.
-   */
-  public static <T extends Comparable<T> & Serializable> SerializableMatcher<T> lessThanOrEqualTo(
-      final T target) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.lessThanOrEqualTo(target);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#lessThanOrEqualTo()}.
-   *
-   * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}.
-   * It is explicitly <i>not</i> required or expected to be serializable via Java serialization.
-   */
-  public static <T extends Comparable<T>> SerializableMatcher<T> lessThanOrEqualTo(
-      Coder<T> coder, T target) {
-    final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target);
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.lessThanOrEqualTo(targetSupplier.get());
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#not}.
-   */
-  public static <T> SerializableMatcher<T> not(final SerializableMatcher<T> matcher) {
-    return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-      @Override
-      public Matcher<T> get() {
-        return Matchers.not(matcher);
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to
-   * {@link Matchers#nullValue}.
-   */
-  public static SerializableMatcher<Object> nullValue() {
-    return fromSupplier(new SerializableSupplier<Matcher<Object>>() {
-      @Override
-      public Matcher<Object> get() {
-        return Matchers.nullValue();
-      }
-    });
-  }
-
-  /**
-   * A {@link SerializableMatcher} with identical criteria to {@link Matchers#startsWith}.
-   */
-  public static SerializableMatcher<String> startsWith(final String substring) {
-    return fromSupplier(new SerializableSupplier<Matcher<String>>() {
-      @Override
-      public Matcher<String> get() {
-        return Matchers.startsWith(substring);
-      }
-    });
-  }
-
-  private static class KvKeyMatcher<K, V>
-  extends BaseMatcher<KV<? extends K, ? extends V>>
-  implements SerializableMatcher<KV<? extends K, ? extends V>> {
-    private final SerializableMatcher<? super K> keyMatcher;
-
-    public KvKeyMatcher(SerializableMatcher<? super K> keyMatcher) {
-      this.keyMatcher = keyMatcher;
-    }
-
-    @Override
-    public boolean matches(Object item) {
-      @SuppressWarnings("unchecked")
-      KV<K, ?> kvItem = (KV<K, ?>) item;
-      return keyMatcher.matches(kvItem.getKey());
-    }
-
-    @Override
-    public void describeMismatch(Object item, Description mismatchDescription) {
-      @SuppressWarnings("unchecked")
-      KV<K, ?> kvItem = (KV<K, ?>) item;
-      if (!keyMatcher.matches(kvItem.getKey())) {
-        mismatchDescription.appendText("key did not match: ");
-        keyMatcher.describeMismatch(kvItem.getKey(), mismatchDescription);
-      }
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("KV with key matching ");
-      keyMatcher.describeTo(description);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .addValue(keyMatcher)
-          .toString();
-    }
-  }
-
-  private static class KvValueMatcher<K, V>
-  extends BaseMatcher<KV<? extends K, ? extends V>>
-  implements SerializableMatcher<KV<? extends K, ? extends V>> {
-    private final SerializableMatcher<? super V> valueMatcher;
-
-    public KvValueMatcher(SerializableMatcher<? super V> valueMatcher) {
-      this.valueMatcher = valueMatcher;
-    }
-
-    @Override
-    public boolean matches(Object item) {
-      @SuppressWarnings("unchecked")
-      KV<?, V> kvItem = (KV<?, V>) item;
-      return valueMatcher.matches(kvItem.getValue());
-    }
-
-    @Override
-    public void describeMismatch(Object item, Description mismatchDescription) {
-      @SuppressWarnings("unchecked")
-      KV<?, V> kvItem = (KV<?, V>) item;
-      if (!valueMatcher.matches(kvItem.getValue())) {
-        mismatchDescription.appendText("value did not match: ");
-        valueMatcher.describeMismatch(kvItem.getValue(), mismatchDescription);
-      }
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("KV with value matching ");
-      valueMatcher.describeTo(description);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .addValue(valueMatcher)
-          .toString();
-    }
-  }
-
-  /**
-   * Constructs a {@link SerializableMatcher} from a non-serializable {@link Matcher} via
-   * indirection through {@link SerializableSupplier}.
-   *
-   * <p>To wrap a {@link Matcher} which is not serializable, provide a {@link SerializableSupplier}
-   * with a {@link SerializableSupplier#get()} method that returns a fresh instance of the
-   * {@link Matcher} desired. The resulting {@link SerializableMatcher} will behave according to
-   * the {@link Matcher} returned by {@link SerializableSupplier#get() get()} when it is invoked
-   * during matching (which may occur on another machine, such as a Dataflow worker).
-   *
-   * <code>
-   * return fromSupplier(new SerializableSupplier<Matcher<T>>() {
-   *   *     @Override
-   *     public Matcher<T> get() {
-   *       return new MyMatcherForT();
-   *     }
-   * });
-   * </code>
-   */
-  public static <T> SerializableMatcher<T> fromSupplier(
-      SerializableSupplier<Matcher<T>> supplier) {
-    return new SerializableMatcherFromSupplier<>(supplier);
-  }
-
-  /**
-   * Supplies values of type {@code T}, and is serializable. Thus, even if {@code T} is not
-   * serializable, the supplier can be serialized and provide a {@code T} wherever it is
-   * deserialized.
-   *
-   * @param <T> the type of value supplied.
-   */
-  public interface SerializableSupplier<T> extends Serializable {
-    T get();
-  }
-
-  /**
-   * Since the delegate {@link Matcher} is not generally serializable, instead this takes a nullary
-   * SerializableFunction to return such a matcher.
-   */
-  private static class SerializableMatcherFromSupplier<T> extends BaseMatcher<T>
-  implements SerializableMatcher<T> {
-
-    private SerializableSupplier<Matcher<T>> supplier;
-
-    public SerializableMatcherFromSupplier(SerializableSupplier<Matcher<T>> supplier) {
-      this.supplier = supplier;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      supplier.get().describeTo(description);
-    }
-
-    @Override
-    public boolean matches(Object item) {
-      return supplier.get().matches(item);
-    }
-
-    @Override
-    public void describeMismatch(Object item, Description mismatchDescription) {
-      supplier.get().describeMismatch(item, mismatchDescription);
-    }
-  }
-
-  /**
-   * Wraps any value that can be encoded via a {@link Coder} to make it {@link Serializable}.
-   * This is not likely to be a good encoding, so should be used only for tests, where data
-   * volume is small and minor costs are not critical.
-   */
-  private static class SerializableViaCoder<T> implements SerializableSupplier<T> {
-    /** Cached value that is not serialized. */
-    @Nullable
-    private transient T value;
-
-    /** The bytes of {@link #value} when encoded via {@link #coder}. */
-    private byte[] encodedValue;
-
-    private Coder<T> coder;
-
-    public SerializableViaCoder(Coder<T> coder, T value) {
-      this.coder = coder;
-      this.value = value;
-      try {
-        this.encodedValue = CoderUtils.encodeToByteArray(coder, value);
-      } catch (CoderException exc) {
-        throw new RuntimeException("Error serializing via Coder", exc);
-      }
-    }
-
-    @Override
-    public T get() {
-      if (value == null) {
-        try {
-          value = CoderUtils.decodeFromByteArray(coder, encodedValue);
-        } catch (CoderException exc) {
-          throw new RuntimeException("Error deserializing via Coder", exc);
-        }
-      }
-      return value;
-    }
-  }
-
-  /**
-   * Wraps any array with values that can be encoded via a {@link Coder} to make it
-   * {@link Serializable}. This is not likely to be a good encoding, so should be used only for
-   * tests, where data volume is small and minor costs are not critical.
-   */
-  private static class SerializableArrayViaCoder<T> implements SerializableSupplier<T[]> {
-    /** Cached value that is not serialized. */
-    @Nullable
-    private transient T[] value;
-
-    /** The bytes of {@link #value} when encoded via {@link #coder}. */
-    private byte[] encodedValue;
-
-    private Coder<List<T>> coder;
-
-    public SerializableArrayViaCoder(Coder<T> elementCoder, T[] value) {
-      this.coder = ListCoder.of(elementCoder);
-      this.value = value;
-      try {
-        this.encodedValue = CoderUtils.encodeToByteArray(coder, Arrays.asList(value));
-      } catch (CoderException exc) {
-        throw UserCodeException.wrap(exc);
-      }
-    }
-
-    @Override
-    public T[] get() {
-      if (value == null) {
-        try {
-          @SuppressWarnings("unchecked")
-          T[] decoded = (T[]) CoderUtils.decodeFromByteArray(coder, encodedValue).toArray();
-          value = decoded;
-        } catch (CoderException exc) {
-          throw new RuntimeException("Error deserializing via Coder", exc);
-        }
-      }
-      return value;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SourceTestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SourceTestUtils.java
deleted file mode 100644
index feff16a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SourceTestUtils.java
+++ /dev/null
@@ -1,676 +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 com.google.cloud.dataflow.sdk.testing;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/**
- * Helper functions and test harnesses for checking correctness of {@link Source}
- * implementations.
- *
- * <p>Contains a few lightweight utilities (e.g. reading items from a source or a reader,
- * such as {@link #readFromSource} and {@link #readFromUnstartedReader}), as well as
- * heavyweight property testing and stress testing harnesses that help getting a large
- * amount of test coverage with few code. Most notable ones are:
- * <ul>
- *   <li>{@link #assertSourcesEqualReferenceSource} helps testing that the data read
- *   by the union of sources produced by {@link BoundedSource#splitIntoBundles}
- *   is the same as data read by the original source.
- *   <li>If your source implements dynamic work rebalancing, use the
- *   {@code assertSplitAtFraction} family of functions - they test behavior of
- *   {@link BoundedSource.BoundedReader#splitAtFraction}, in particular, that
- *   various consistency properties are respected and the total set of data read
- *   by the source is preserved when splits happen.
- *   Use {@link #assertSplitAtFractionBehavior} to test individual cases
- *   of {@code splitAtFraction} and use {@link #assertSplitAtFractionExhaustive}
- *   as a heavy-weight stress test including concurrency. We strongly recommend to
- *   use both.
- * </ul>
- * For example usages, see the unit tests of classes such as
- * {@link com.google.cloud.dataflow.sdk.io.AvroSource} or
- * {@link com.google.cloud.dataflow.sdk.io.XmlSource}.
- *
- * <p>Like {@link PAssert}, requires JUnit and Hamcrest to be present in the classpath.
- */
-public class SourceTestUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(SourceTestUtils.class);
-
-  // A wrapper around a value of type T that compares according to the structural
-  // value provided by a Coder<T>, but prints both the original and structural value,
-  // to help get good error messages from JUnit equality assertion failures and such.
-  private static class ReadableStructuralValue<T> {
-    private T originalValue;
-    private Object structuralValue;
-
-    public ReadableStructuralValue(T originalValue, Object structuralValue) {
-      this.originalValue = originalValue;
-      this.structuralValue = structuralValue;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(structuralValue);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (obj == null || !(obj instanceof ReadableStructuralValue)) {
-        return false;
-      }
-      return Objects.equals(structuralValue, ((ReadableStructuralValue) obj).structuralValue);
-    }
-
-    @Override
-    public String toString() {
-      return String.format("[%s (structural %s)]", originalValue, structuralValue);
-    }
-  }
-
-  /**
-   * Testing utilities below depend on standard assertions and matchers to compare elements read by
-   * sources. In general the elements may not implement {@code equals}/{@code hashCode} properly,
-   * however every source has a {@link Coder} and every {@code Coder} can
-   * produce a {@link Coder#structuralValue} whose {@code equals}/{@code hashCode} is
-   * consistent with equality of encoded format.
-   * So we use this {@link Coder#structuralValue} to compare elements read by sources.
-   */
-  public static <T> List<ReadableStructuralValue<T>> createStructuralValues(
-      Coder<T> coder, List<T> list)
-      throws Exception {
-    List<ReadableStructuralValue<T>> result = new ArrayList<>();
-    for (T elem : list) {
-      result.add(new ReadableStructuralValue<>(elem, coder.structuralValue(elem)));
-    }
-    return result;
-  }
-
-  /**
-   * Reads all elements from the given {@link BoundedSource}.
-   */
-  public static <T> List<T> readFromSource(BoundedSource<T> source, PipelineOptions options)
-      throws IOException {
-    try (BoundedSource.BoundedReader<T> reader = source.createReader(options)) {
-      return readFromUnstartedReader(reader);
-    }
-  }
-
-  /**
-   * Reads all elements from the given unstarted {@link Source.Reader}.
-   */
-  public static <T> List<T> readFromUnstartedReader(Source.Reader<T> reader) throws IOException {
-    return readRemainingFromReader(reader, false);
-  }
-
-  /**
-   * Reads all elements from the given started {@link Source.Reader}.
-   */
-  public static <T> List<T> readFromStartedReader(Source.Reader<T> reader) throws IOException {
-    return readRemainingFromReader(reader, true);
-  }
-
-  /**
-   * Read elements from a {@link Source.Reader} until n elements are read.
-   */
-  public static <T> List<T> readNItemsFromUnstartedReader(Source.Reader<T> reader, int n)
-      throws IOException {
-    return readNItemsFromReader(reader, n, false);
-  }
-
-  /**
-   * Read elements from a {@link Source.Reader} that has already had {@link Source.Reader#start}
-   * called on it, until n elements are read.
-   */
-  public static <T> List<T> readNItemsFromStartedReader(Source.Reader<T> reader, int n)
-      throws IOException {
-    return readNItemsFromReader(reader, n, true);
-  }
-
-  /**
-   * Read elements from a {@link Source.Reader} until n elements are read.
-   *
-   * <p>There must be at least n elements remaining in the reader, except for
-   * the case when n is {@code Integer.MAX_VALUE}, which means "read all
-   * remaining elements".
-   */
-  private static <T> List<T> readNItemsFromReader(Source.Reader<T> reader, int n, boolean started)
-      throws IOException {
-    List<T> res = new ArrayList<>();
-    for (int i = 0; i < n; i++) {
-      boolean shouldStart = (i == 0 && !started);
-      boolean more = shouldStart ? reader.start() : reader.advance();
-      if (n != Integer.MAX_VALUE) {
-        assertTrue(more);
-      }
-      if (!more) {
-        break;
-      }
-      res.add(reader.getCurrent());
-    }
-    return res;
-  }
-
-  /**
-   * Read all remaining elements from a {@link Source.Reader}.
-   */
-  public static <T> List<T> readRemainingFromReader(Source.Reader<T> reader, boolean started)
-      throws IOException {
-    return readNItemsFromReader(reader, Integer.MAX_VALUE, started);
-  }
-
-  /**
-   * Given a reference {@code Source} and a list of {@code Source}s, assert that the union of
-   * the records read from the list of sources is equal to the records read from the reference
-   * source.
-   */
-  public static <T> void assertSourcesEqualReferenceSource(
-      BoundedSource<T> referenceSource,
-      List<? extends BoundedSource<T>> sources,
-      PipelineOptions options)
-      throws Exception {
-    Coder<T> coder = referenceSource.getDefaultOutputCoder();
-    List<T> referenceRecords = readFromSource(referenceSource, options);
-    List<T> bundleRecords = new ArrayList<>();
-    for (BoundedSource<T> source : sources) {
-      assertThat(
-          "Coder type for source "
-              + source
-              + " is not compatible with Coder type for referenceSource "
-              + referenceSource,
-          source.getDefaultOutputCoder(),
-          equalTo(coder));
-      List<T> elems = readFromSource(source, options);
-      bundleRecords.addAll(elems);
-    }
-    List<ReadableStructuralValue<T>> bundleValues =
-        createStructuralValues(coder, bundleRecords);
-    List<ReadableStructuralValue<T>> referenceValues =
-        createStructuralValues(coder, referenceRecords);
-    assertThat(bundleValues, containsInAnyOrder(referenceValues.toArray()));
-  }
-
-  /**
-   * Assert that a {@code Reader} returns a {@code Source} that, when read from, produces the same
-   * records as the reader.
-   */
-  public static <T> void assertUnstartedReaderReadsSameAsItsSource(
-      BoundedSource.BoundedReader<T> reader, PipelineOptions options) throws Exception {
-    Coder<T> coder = reader.getCurrentSource().getDefaultOutputCoder();
-    List<T> expected = readFromUnstartedReader(reader);
-    List<T> actual = readFromSource(reader.getCurrentSource(), options);
-    List<ReadableStructuralValue<T>> expectedStructural = createStructuralValues(coder, expected);
-    List<ReadableStructuralValue<T>> actualStructural = createStructuralValues(coder, actual);
-    assertThat(actualStructural, containsInAnyOrder(expectedStructural.toArray()));
-  }
-
-  /**
-   * Expected outcome of
-   * {@link com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader#splitAtFraction}.
-   */
-  public enum ExpectedSplitOutcome {
-    /**
-     * The operation must succeed and the results must be consistent.
-     */
-    MUST_SUCCEED_AND_BE_CONSISTENT,
-    /**
-     * The operation must fail (return {@code null}).
-     */
-    MUST_FAIL,
-    /**
-     * The operation must either fail, or succeed and the results be consistent.
-     */
-    MUST_BE_CONSISTENT_IF_SUCCEEDS
-  }
-
-  /**
-   * Contains two values: the number of items in the primary source, and the number of items in
-   * the residual source, -1 if split failed.
-   */
-  private static class SplitAtFractionResult {
-    public int numPrimaryItems;
-    public int numResidualItems;
-
-    public SplitAtFractionResult(int numPrimaryItems, int numResidualItems) {
-      this.numPrimaryItems = numPrimaryItems;
-      this.numResidualItems = numResidualItems;
-    }
-  }
-
-  /**
-   * Asserts that the {@code source}'s reader either fails to {@code splitAtFraction(fraction)}
-   * after reading {@code numItemsToReadBeforeSplit} items, or succeeds in a way that is
-   * consistent according to {@link #assertSplitAtFractionSucceedsAndConsistent}.
-   * <p> Returns SplitAtFractionResult.
-   */
-
-  public static <T> SplitAtFractionResult assertSplitAtFractionBehavior(
-      BoundedSource<T> source,
-      int numItemsToReadBeforeSplit,
-      double splitFraction,
-      ExpectedSplitOutcome expectedOutcome,
-      PipelineOptions options)
-      throws Exception {
-    return assertSplitAtFractionBehaviorImpl(
-        source, readFromSource(source, options), numItemsToReadBeforeSplit, splitFraction,
-        expectedOutcome, options);
-  }
-
-  /**
-   * Compares two lists elementwise and throws a detailed assertion failure optimized for
-   * human reading in case they are unequal.
-   */
-  private static <T> void assertListsEqualInOrder(
-      String message, String expectedLabel, List<T> expected, String actualLabel, List<T> actual) {
-    int i = 0;
-    for (; i < expected.size() && i < actual.size(); ++i) {
-      if (!Objects.equals(expected.get(i), actual.get(i))) {
-        Assert.fail(String.format(
-            "%s: %s and %s have %d items in common and then differ. "
-            + "Item in %s (%d more): %s, item in %s (%d more): %s",
-            message, expectedLabel, actualLabel, i,
-            expectedLabel, expected.size() - i - 1, expected.get(i),
-            actualLabel, actual.size() - i - 1, actual.get(i)));
-      }
-    }
-    if (i < expected.size() /* but i == actual.size() */) {
-      Assert.fail(String.format(
-          "%s: %s has %d more items after matching all %d from %s. First 5: %s",
-          message, expectedLabel, expected.size() - actual.size(), actual.size(), actualLabel,
-          expected.subList(actual.size(), Math.min(expected.size(), actual.size() + 5))));
-    } else if (i < actual.size() /* but i == expected.size() */) {
-      Assert.fail(String.format(
-          "%s: %s has %d more items after matching all %d from %s. First 5: %s",
-          message, actualLabel, actual.size() - expected.size(), expected.size(), expectedLabel,
-          actual.subList(expected.size(), Math.min(actual.size(), expected.size() + 5))));
-    } else {
-      // All is well.
-    }
-  }
-
-  private static <T> SourceTestUtils.SplitAtFractionResult assertSplitAtFractionBehaviorImpl(
-      BoundedSource<T> source, List<T> expectedItems, int numItemsToReadBeforeSplit,
-      double splitFraction, ExpectedSplitOutcome expectedOutcome, PipelineOptions options)
-      throws Exception {
-    try (BoundedSource.BoundedReader<T> reader = source.createReader(options)) {
-      BoundedSource<T> originalSource = reader.getCurrentSource();
-      List<T> currentItems = readNItemsFromUnstartedReader(reader, numItemsToReadBeforeSplit);
-      BoundedSource<T> residual = reader.splitAtFraction(splitFraction);
-      if (residual != null) {
-        assertFalse(
-            String.format(
-                "Primary source didn't change after a successful split of %s at %f "
-                + "after reading %d items. "
-                + "Was the source object mutated instead of creating a new one? "
-                + "Source objects MUST be immutable.",
-                source, splitFraction, numItemsToReadBeforeSplit),
-            reader.getCurrentSource() == originalSource);
-        assertFalse(
-            String.format(
-                "Residual source equal to original source after a successful split of %s at %f "
-                + "after reading %d items. "
-                + "Was the source object mutated instead of creating a new one? "
-                + "Source objects MUST be immutable.",
-                source, splitFraction, numItemsToReadBeforeSplit),
-            reader.getCurrentSource() == residual);
-      }
-      // Failure cases are: must succeed but fails; must fail but succeeds.
-      switch (expectedOutcome) {
-        case MUST_SUCCEED_AND_BE_CONSISTENT:
-          assertNotNull(
-              "Failed to split reader of source: "
-                  + source
-                  + " at "
-                  + splitFraction
-                  + " after reading "
-                  + numItemsToReadBeforeSplit
-                  + " items",
-              residual);
-          break;
-        case MUST_FAIL:
-          assertEquals(null, residual);
-          break;
-        case MUST_BE_CONSISTENT_IF_SUCCEEDS:
-          // Nothing.
-          break;
-      }
-      currentItems.addAll(readRemainingFromReader(reader, numItemsToReadBeforeSplit > 0));
-      BoundedSource<T> primary = reader.getCurrentSource();
-      return verifySingleSplitAtFractionResult(
-          source, expectedItems, currentItems, primary, residual,
-          numItemsToReadBeforeSplit, splitFraction, options);
-    }
-  }
-
-  private static <T> SourceTestUtils.SplitAtFractionResult verifySingleSplitAtFractionResult(
-      BoundedSource<T> source, List<T> expectedItems, List<T> currentItems,
-      BoundedSource<T> primary, BoundedSource<T> residual,
-      int numItemsToReadBeforeSplit, double splitFraction, PipelineOptions options)
-      throws Exception {
-    List<T> primaryItems = readFromSource(primary, options);
-    if (residual != null) {
-      List<T> residualItems = readFromSource(residual, options);
-      List<T> totalItems = new ArrayList<>();
-      totalItems.addAll(primaryItems);
-      totalItems.addAll(residualItems);
-      String errorMsgForPrimarySourceComp =
-          String.format(
-              "Continued reading after split yielded different items than primary source: "
-                  + "split at %s after reading %s items, original source: %s, primary source: %s",
-              splitFraction,
-              numItemsToReadBeforeSplit,
-              source,
-              primary);
-      String errorMsgForTotalSourceComp =
-          String.format(
-              "Items in primary and residual sources after split do not add up to items "
-                  + "in the original source. Split at %s after reading %s items; "
-                  + "original source: %s, primary: %s, residual: %s",
-              splitFraction,
-              numItemsToReadBeforeSplit,
-              source,
-              primary,
-              residual);
-      Coder<T> coder = primary.getDefaultOutputCoder();
-      List<ReadableStructuralValue<T>> primaryValues =
-          createStructuralValues(coder, primaryItems);
-      List<ReadableStructuralValue<T>> currentValues =
-          createStructuralValues(coder, currentItems);
-      List<ReadableStructuralValue<T>> expectedValues =
-          createStructuralValues(coder, expectedItems);
-      List<ReadableStructuralValue<T>> totalValues =
-          createStructuralValues(coder, totalItems);
-      assertListsEqualInOrder(
-          errorMsgForPrimarySourceComp, "current", currentValues, "primary", primaryValues);
-      assertListsEqualInOrder(
-          errorMsgForTotalSourceComp, "total", expectedValues, "primary+residual", totalValues);
-      return new SplitAtFractionResult(primaryItems.size(), residualItems.size());
-    }
-    return new SplitAtFractionResult(primaryItems.size(), -1);
-  }
-
-  /**
-   * Verifies some consistency properties of
-   * {@link BoundedSource.BoundedReader#splitAtFraction} on the given source. Equivalent to
-   * the following pseudocode:
-   * <pre>
-   *   Reader reader = source.createReader();
-   *   read N items from reader;
-   *   Source residual = reader.splitAtFraction(splitFraction);
-   *   Source primary = reader.getCurrentSource();
-   *   assert: items in primary == items we read so far
-   *                               + items we'll get by continuing to read from reader;
-   *   assert: items in original source == items in primary + items in residual
-   * </pre>
-   */
-  public static <T> void assertSplitAtFractionSucceedsAndConsistent(
-      BoundedSource<T> source,
-      int numItemsToReadBeforeSplit,
-      double splitFraction,
-      PipelineOptions options)
-      throws Exception {
-    assertSplitAtFractionBehavior(
-        source,
-        numItemsToReadBeforeSplit,
-        splitFraction,
-        ExpectedSplitOutcome.MUST_SUCCEED_AND_BE_CONSISTENT,
-        options);
-  }
-
-  /**
-   * Asserts that the {@code source}'s reader fails to {@code splitAtFraction(fraction)}
-   * after reading {@code numItemsToReadBeforeSplit} items.
-   */
-  public static <T> void assertSplitAtFractionFails(
-      BoundedSource<T> source,
-      int numItemsToReadBeforeSplit,
-      double splitFraction,
-      PipelineOptions options)
-      throws Exception {
-    assertSplitAtFractionBehavior(
-        source, numItemsToReadBeforeSplit, splitFraction, ExpectedSplitOutcome.MUST_FAIL, options);
-  }
-
-  private static class SplitFractionStatistics {
-    List<Double> successfulFractions = new ArrayList<>();
-    List<Double> nonTrivialFractions = new ArrayList<>();
-  }
-
-  /**
-   * Asserts that given a start position,
-   * {@link BoundedSource.BoundedReader#splitAtFraction} at every interesting fraction (halfway
-   * between two fractions that differ by at least one item) can be called successfully and the
-   * results are consistent if a split succeeds.
-   */
-  private static <T> void assertSplitAtFractionBinary(
-      BoundedSource<T> source,
-      List<T> expectedItems,
-      int numItemsToBeReadBeforeSplit,
-      double leftFraction,
-      SplitAtFractionResult leftResult,
-      double rightFraction,
-      SplitAtFractionResult rightResult,
-      PipelineOptions options,
-      SplitFractionStatistics stats)
-      throws Exception {
-    if (rightFraction - leftFraction < 0.001) {
-      // Do not recurse too deeply. Otherwise we will end up in infinite
-      // recursion, e.g., while trying to find the exact minimal fraction s.t.
-      // split succeeds. A precision of 0.001 when looking for such a fraction
-      // ought to be enough for everybody.
-      return;
-    }
-    double middleFraction = (rightFraction + leftFraction) / 2;
-    if (leftResult == null) {
-      leftResult = assertSplitAtFractionBehaviorImpl(
-          source, expectedItems, numItemsToBeReadBeforeSplit, leftFraction,
-          ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
-    }
-    if (rightResult == null) {
-      rightResult = assertSplitAtFractionBehaviorImpl(
-          source, expectedItems, numItemsToBeReadBeforeSplit, rightFraction,
-          ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
-    }
-    SplitAtFractionResult middleResult = assertSplitAtFractionBehaviorImpl(
-        source, expectedItems, numItemsToBeReadBeforeSplit, middleFraction,
-        ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options);
-    if (middleResult.numResidualItems != -1) {
-      stats.successfulFractions.add(middleFraction);
-    }
-    if (middleResult.numResidualItems > 0) {
-      stats.nonTrivialFractions.add(middleFraction);
-    }
-    // Two split fractions are equivalent if they yield the same number of
-    // items in primary vs. residual source. Left and right are already not
-    // equivalent. Recurse into [left, middle) and [right, middle) respectively
-    // if middle is not equivalent to left or right.
-    if (leftResult.numPrimaryItems != middleResult.numPrimaryItems) {
-      assertSplitAtFractionBinary(
-          source, expectedItems, numItemsToBeReadBeforeSplit,
-          leftFraction, leftResult, middleFraction, middleResult, options, stats);
-    }
-    if (rightResult.numPrimaryItems != middleResult.numPrimaryItems) {
-      assertSplitAtFractionBinary(
-          source, expectedItems, numItemsToBeReadBeforeSplit,
-          middleFraction, middleResult, rightFraction, rightResult, options, stats);
-    }
-  }
-
-  private static final int MAX_CONCURRENT_SPLITTING_TRIALS_PER_ITEM = 100;
-  private static final int MAX_CONCURRENT_SPLITTING_TRIALS_TOTAL = 1000;
-
-  /**
-   * Asserts that for each possible start position,
-   * {@link BoundedSource.BoundedReader#splitAtFraction} at every interesting fraction (halfway
-   * between two fractions that differ by at least one item) can be called successfully and the
-   * results are consistent if a split succeeds. Verifies multithreaded splitting as well.
-   */
-  public static <T> void assertSplitAtFractionExhaustive(
-      BoundedSource<T> source, PipelineOptions options) throws Exception {
-    List<T> expectedItems = readFromSource(source, options);
-    assertFalse("Empty source", expectedItems.isEmpty());
-    assertFalse("Source reads a single item", expectedItems.size() == 1);
-    List<List<Double>> allNonTrivialFractions = new ArrayList<>();
-    {
-      boolean anySuccessfulFractions = false;
-      boolean anyNonTrivialFractions = false;
-      for (int i = 0; i < expectedItems.size(); i++) {
-        SplitFractionStatistics stats = new SplitFractionStatistics();
-        assertSplitAtFractionBinary(source, expectedItems, i,
-            0.0, null, 1.0, null, options, stats);
-        if (!stats.successfulFractions.isEmpty()) {
-          anySuccessfulFractions = true;
-        }
-        if (!stats.nonTrivialFractions.isEmpty()) {
-          anyNonTrivialFractions = true;
-        }
-        allNonTrivialFractions.add(stats.nonTrivialFractions);
-      }
-      assertTrue(
-          "splitAtFraction test completed vacuously: no successful split fractions found",
-          anySuccessfulFractions);
-      assertTrue(
-          "splitAtFraction test completed vacuously: no non-trivial split fractions found",
-          anyNonTrivialFractions);
-    }
-    {
-      // Perform a stress test of "racy" concurrent splitting:
-      // for every position (number of items read), try to split at the minimum nontrivial
-      // split fraction for that position concurrently with reading the record at that position.
-      // To ensure that the test is non-vacuous, make sure that the splitting succeeds
-      // at least once and fails at least once.
-      ExecutorService executor = Executors.newFixedThreadPool(2);
-      int numTotalTrials = 0;
-      for (int i = 0; i < expectedItems.size(); i++) {
-        double minNonTrivialFraction = 2.0;  // Greater than any possible fraction.
-        for (double fraction : allNonTrivialFractions.get(i)) {
-          minNonTrivialFraction = Math.min(minNonTrivialFraction, fraction);
-        }
-        if (minNonTrivialFraction == 2.0) {
-          // This will not happen all the time because otherwise the test above would
-          // detect vacuousness.
-          continue;
-        }
-        int numTrials = 0;
-        boolean haveSuccess = false, haveFailure = false;
-        while (true) {
-          ++numTrials;
-          if (numTrials > MAX_CONCURRENT_SPLITTING_TRIALS_PER_ITEM) {
-            LOG.warn(
-                "After {} concurrent splitting trials at item #{}, observed only {}, "
-                + "giving up on this item",
-                numTrials, i, haveSuccess ? "success" : "failure");
-            break;
-          }
-          if (assertSplitAtFractionConcurrent(
-              executor, source, expectedItems, i, minNonTrivialFraction, options)) {
-            haveSuccess = true;
-          } else {
-            haveFailure = true;
-          }
-          if (haveSuccess && haveFailure) {
-            LOG.info(
-                "{} trials to observe both success and failure of concurrent splitting at item #{}",
-                numTrials, i);
-            break;
-          }
-        }
-        numTotalTrials += numTrials;
-        if (numTotalTrials > MAX_CONCURRENT_SPLITTING_TRIALS_TOTAL) {
-          LOG.warn(
-              "After {} total concurrent splitting trials, considered only {} items, giving up.",
-              numTotalTrials, i);
-          break;
-        }
-      }
-      LOG.info(
-          "{} total concurrent splitting trials for {} items",
-          numTotalTrials, expectedItems.size());
-    }
-  }
-
-  private static <T> boolean assertSplitAtFractionConcurrent(
-      ExecutorService executor, BoundedSource<T> source, List<T> expectedItems,
-      final int numItemsToReadBeforeSplitting, final double fraction, PipelineOptions options)
-      throws Exception {
-    @SuppressWarnings("resource")  // Closed in readerThread
-    final BoundedSource.BoundedReader<T> reader = source.createReader(options);
-    final CountDownLatch unblockSplitter = new CountDownLatch(1);
-    Future<List<T>> readerThread =
-        executor.submit(
-            new Callable<List<T>>() {
-              @Override
-              public List<T> call() throws Exception {
-                try {
-                  List<T> items =
-                      readNItemsFromUnstartedReader(reader, numItemsToReadBeforeSplitting);
-                  unblockSplitter.countDown();
-                  items.addAll(readRemainingFromReader(reader, numItemsToReadBeforeSplitting > 0));
-                  return items;
-                } finally {
-                  reader.close();
-                }
-              }
-            });
-    Future<KV<BoundedSource<T>, BoundedSource<T>>> splitterThread = executor.submit(
-        new Callable<KV<BoundedSource<T>, BoundedSource<T>>>() {
-          @Override
-          public KV<BoundedSource<T>, BoundedSource<T>> call() throws Exception {
-            unblockSplitter.await();
-            BoundedSource<T> residual = reader.splitAtFraction(fraction);
-            if (residual == null) {
-              return null;
-            }
-            return KV.of(reader.getCurrentSource(), residual);
-          }
-        });
-    List<T> currentItems = readerThread.get();
-    KV<BoundedSource<T>, BoundedSource<T>> splitSources = splitterThread.get();
-    if (splitSources == null) {
-      return false;
-    }
-    SplitAtFractionResult res = verifySingleSplitAtFractionResult(
-        source, expectedItems, currentItems, splitSources.getKey(), splitSources.getValue(),
-        numItemsToReadBeforeSplitting, fraction, options);
-    return (res.numResidualItems > 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
deleted file mode 100644
index 17b1538..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
+++ /dev/null
@@ -1,205 +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 com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.util.TestCredential;
-import com.google.common.base.Optional;
-import com.google.common.base.Strings;
-import com.google.common.collect.Iterators;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.Iterator;
-
-import javax.annotation.Nullable;
-
-/**
- * A creator of test pipelines that can be used inside of tests that can be
- * configured to run locally or against a remote pipeline runner.
- *
- * <p>It is recommended to tag hand-selected tests for this purpose using the
- * {@link RunnableOnService} {@link Category} annotation, as each test run against a pipeline runner
- * will utilize resources of that pipeline runner.
- *
- * <p>In order to run tests on a pipeline runner, the following conditions must be met:
- * <ul>
- *   <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
- *   options. For example:
- *   <pre>{@code [
- *     "--runner=com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineRunner",
- *     "--project=mygcpproject",
- *     "--stagingLocation=gs://mygcsbucket/path"
- *     ]}</pre>
- *     Note that the set of pipeline options required is pipeline runner specific.
- *   </li>
- *   <li>Jars containing the SDK and test classes must be available on the classpath.</li>
- * </ul>
- *
- * <p>Use {@link PAssert} for tests, as it integrates with this test harness in both direct and
- * remote execution modes. For example:
- * <pre>{@code
- * Pipeline p = TestPipeline.create();
- * PCollection<Integer> output = ...
- *
- * PAssert.that(output)
- *     .containsInAnyOrder(1, 2, 3, 4);
- * p.run();
- * }</pre>
- *
- * <p>For pipeline runners, it is required that they must throw an {@link AssertionError}
- * containing the message from the {@link PAssert} that failed.
- */
-public class TestPipeline extends Pipeline {
-  private static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions";
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-
-  /**
-   * Creates and returns a new test pipeline.
-   *
-   * <p>Use {@link PAssert} to add tests, then call
-   * {@link Pipeline#run} to execute the pipeline and check the tests.
-   */
-  public static TestPipeline create() {
-    return fromOptions(testingPipelineOptions());
-  }
-
-  public static TestPipeline fromOptions(PipelineOptions options) {
-    return new TestPipeline(PipelineRunner.fromOptions(options), options);
-  }
-
-  private TestPipeline(PipelineRunner<? extends PipelineResult> runner, PipelineOptions options) {
-    super(runner, options);
-  }
-
-  /**
-   * Runs this {@link TestPipeline}, unwrapping any {@code AssertionError}
-   * that is raised during testing.
-   */
-  @Override
-  public PipelineResult run() {
-    try {
-      return super.run();
-    } catch (RuntimeException exc) {
-      Throwable cause = exc.getCause();
-      if (cause instanceof AssertionError) {
-        throw (AssertionError) cause;
-      } else {
-        throw exc;
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "TestPipeline#" + getOptions().as(ApplicationNameOptions.class).getAppName();
-  }
-
-  /**
-   * Creates {@link PipelineOptions} for testing.
-   */
-  public static PipelineOptions testingPipelineOptions() {
-    try {
-      @Nullable String beamTestPipelineOptions =
-          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS);
-
-      PipelineOptions options =
-          Strings.isNullOrEmpty(beamTestPipelineOptions)
-              ? PipelineOptionsFactory.create()
-              : PipelineOptionsFactory.fromArgs(
-                      MAPPER.readValue(
-                          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), String[].class))
-                  .as(PipelineOptions.class);
-
-      options.as(ApplicationNameOptions.class).setAppName(getAppName());
-      // If no options were specified, use a test credential object on all pipelines.
-      if (Strings.isNullOrEmpty(beamTestPipelineOptions)) {
-        options.as(GcpOptions.class).setGcpCredential(new TestCredential());
-      }
-      options.setStableUniqueNames(CheckEnabled.ERROR);
-      return options;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to instantiate test options from system property "
-          + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS + ":"
-          + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), e);
-    }
-  }
-
-  /** Returns the class + method name of the test, or a default name. */
-  private static String getAppName() {
-    Optional<StackTraceElement> stackTraceElement = findCallersStackTrace();
-    if (stackTraceElement.isPresent()) {
-      String methodName = stackTraceElement.get().getMethodName();
-      String className = stackTraceElement.get().getClassName();
-      if (className.contains(".")) {
-        className = className.substring(className.lastIndexOf(".") + 1);
-      }
-      return className + "-" + methodName;
-    }
-    return "UnitTest";
-  }
-
-  /** Returns the {@link StackTraceElement} of the calling class. */
-  private static Optional<StackTraceElement> findCallersStackTrace() {
-    Iterator<StackTraceElement> elements =
-        Iterators.forArray(Thread.currentThread().getStackTrace());
-    // First find the TestPipeline class in the stack trace.
-    while (elements.hasNext()) {
-      StackTraceElement next = elements.next();
-      if (TestPipeline.class.getName().equals(next.getClassName())) {
-        break;
-      }
-    }
-    // Then find the first instance after that is not the TestPipeline
-    Optional<StackTraceElement> firstInstanceAfterTestPipeline = Optional.absent();
-    while (elements.hasNext()) {
-      StackTraceElement next = elements.next();
-      if (!TestPipeline.class.getName().equals(next.getClassName())) {
-        if (!firstInstanceAfterTestPipeline.isPresent()) {
-          firstInstanceAfterTestPipeline = Optional.of(next);
-        }
-        try {
-          Class<?> nextClass = Class.forName(next.getClassName());
-          for (Method method : nextClass.getMethods()) {
-            if (method.getName().equals(next.getMethodName())) {
-              if (method.isAnnotationPresent(org.junit.Test.class)) {
-                return Optional.of(next);
-              } else if (method.isAnnotationPresent(org.junit.Before.class)) {
-                break;
-              }
-            }
-          }
-        } catch (Throwable t) {
-          break;
-        }
-      }
-    }
-    return firstInstanceAfterTestPipeline;
-  }
-}


[16/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java
deleted file mode 100644
index c2264e1..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java
+++ /dev/null
@@ -1,947 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.api.services.datastore.DatastoreV1.PropertyFilter.Operator.EQUAL;
-import static com.google.api.services.datastore.DatastoreV1.PropertyOrder.Direction.DESCENDING;
-import static com.google.api.services.datastore.DatastoreV1.QueryResultBatch.MoreResultsType.NOT_FINISHED;
-import static com.google.api.services.datastore.client.DatastoreHelper.getPropertyMap;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeFilter;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeOrder;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeValue;
-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 org.apache.beam.sdk.annotations.Experimental;
-
-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.api.services.datastore.DatastoreV1.CommitRequest;
-import com.google.api.services.datastore.DatastoreV1.Entity;
-import com.google.api.services.datastore.DatastoreV1.EntityResult;
-import com.google.api.services.datastore.DatastoreV1.Key;
-import com.google.api.services.datastore.DatastoreV1.Key.PathElement;
-import com.google.api.services.datastore.DatastoreV1.PartitionId;
-import com.google.api.services.datastore.DatastoreV1.Query;
-import com.google.api.services.datastore.DatastoreV1.QueryResultBatch;
-import com.google.api.services.datastore.DatastoreV1.RunQueryRequest;
-import com.google.api.services.datastore.DatastoreV1.RunQueryResponse;
-import com.google.api.services.datastore.client.Datastore;
-import com.google.api.services.datastore.client.DatastoreException;
-import com.google.api.services.datastore.client.DatastoreFactory;
-import com.google.api.services.datastore.client.DatastoreHelper;
-import com.google.api.services.datastore.client.DatastoreOptions;
-import com.google.api.services.datastore.client.QuerySplitter;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.EntityCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-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 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#withDataset} and {@link DatastoreIO.Source#withQuery} to
- * specify the dataset to query and the query to read from. You can optionally provide a namespace
- * to query within using {@link DatastoreIO.Source#withNamespace} or a Datastore host using
- * {@link DatastoreIO.Source#withHost}.
- *
- * <p>For example:
- *
- * <pre> {@code
- * // Read a query from Datastore
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String dataset = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection<Entity> entities = p.apply(
- *     Read.from(DatastoreIO.source()
- *         .withDataset(datasetId)
- *         .withQuery(query)
- *         .withHost(host)));
- * } </pre>
- *
- * <p>or:
- *
- * <pre> {@code
- * // Read a query from Datastore using the default namespace and host
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String dataset = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection<Entity> entities = p.apply(DatastoreIO.readFrom(datasetId, 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.api.services.datastore.DatastoreV1.Query.Builder#setLimit(int)}, 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(dataset));
- * p.run();
- * } </pre>
- *
- * <p>To optionally change the host that is used to write to the Datastore, use {@link
- * DatastoreIO#sink} to build a {@link DatastoreIO.Sink} and write to it using the {@link Write}
- * transform:
- *
- * <pre> {@code
- * PCollection<Entity> entities = ...;
- * entities.apply(Write.to(DatastoreIO.sink().withDataset(dataset).withHost(host)));
- * } </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 com.google.cloud.dataflow.sdk.runners.PipelineRunner
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public class DatastoreIO {
-  public static final String DEFAULT_HOST = "https://www.googleapis.com";
-
-  /**
-   * 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 with the default {@code host}.
-   * Configure the {@code dataset}, {@code query}, and {@code namespace} using
-   * {@link DatastoreIO.Source#withDataset}, {@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 with the default {@code host}.
-   * Configure the {@code dataset}, {@code query}, and {@code namespace} using
-   * {@link DatastoreIO.Source#withDataset}, {@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(DEFAULT_HOST, null, null, null);
-  }
-
-  /**
-   * Returns a {@code PTransform} that reads Datastore entities from the query
-   * against the given dataset.
-   */
-  public static Read.Bounded<Entity> readFrom(String datasetId, Query query) {
-    return Read.from(new Source(DEFAULT_HOST, datasetId, query, null));
-  }
-
-  /**
-   * Returns a {@code PTransform} that reads Datastore entities from the query
-   * against the given dataset and host.
-   *
-   * @deprecated prefer {@link #source()} with {@link Source#withHost}, {@link Source#withDataset},
-   *    {@link Source#withQuery}s.
-   */
-  @Deprecated
-  public static Read.Bounded<Entity> readFrom(String host, String datasetId, Query query) {
-    return Read.from(new Source(host, datasetId, 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 getHost() {
-      return host;
-    }
-
-    public String getDataset() {
-      return datasetId;
-    }
-
-    public Query getQuery() {
-      return query;
-    }
-
-    @Nullable
-    public String getNamespace() {
-      return namespace;
-    }
-
-    public Source withDataset(String datasetId) {
-      checkNotNull(datasetId, "datasetId");
-      return new Source(host, datasetId, 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.api.services.datastore.DatastoreV1.Query.Builder#setLimit(int)}, 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() > 0,
-          "Invalid query limit %s: must be positive", query.getLimit());
-      return new Source(host, datasetId, query, namespace);
-    }
-
-    public Source withHost(String host) {
-      checkNotNull(host, "host");
-      return new Source(host, datasetId, query, namespace);
-    }
-
-    public Source withNamespace(@Nullable String namespace) {
-      return new Source(host, datasetId, query, namespace);
-    }
-
-    @Override
-    public Coder<Entity> getDefaultOutputCoder() {
-      return EntityCoder.of();
-    }
-
-    @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(host, datasetId, splitQuery, namespace));
-      }
-      return splits.build();
-    }
-
-    @Override
-    public BoundedReader<Entity> createReader(PipelineOptions pipelineOptions) throws IOException {
-      return new DatastoreReader(this, getDatastore(pipelineOptions));
-    }
-
-    @Override
-    public void validate() {
-      Preconditions.checkNotNull(host, "host");
-      Preconditions.checkNotNull(query, "query");
-      Preconditions.checkNotNull(datasetId, "datasetId");
-    }
-
-    @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(makeFilter(
-          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.getEntityResultCount() == 0) {
-        throw new NoSuchElementException(
-            "Datastore statistics for kind " + ourKind + " unavailable");
-      }
-      Entity entity = batch.getEntityResult(0).getEntity();
-      return getPropertyMap(entity).get("entity_bytes").getIntegerValue();
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(getClass())
-          .add("host", host)
-          .add("dataset", datasetId)
-          .add("query", query)
-          .add("namespace", namespace)
-          .toString();
-    }
-
-    ///////////////////////////////////////////////////////////////////////////////////////////
-
-    private static final Logger LOG = LoggerFactory.getLogger(Source.class);
-    private final String host;
-    /** Not really nullable, but it may be {@code null} for in-progress {@code Source}s. */
-    @Nullable
-    private final String datasetId;
-    /** 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(
-        String host, @Nullable String datasetId, @Nullable Query query,
-        @Nullable String namespace) {
-      this.host = checkNotNull(host, "host");
-      this.datasetId = datasetId;
-      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.setNamespace(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().setNamespace(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(1);
-      RunQueryRequest request = makeRequest(query.build());
-
-      long now = System.currentTimeMillis();
-      RunQueryResponse response = datastore.runQuery(request);
-      LOG.info("Query for latest stats timestamp of dataset {} took {}ms", datasetId,
-          System.currentTimeMillis() - now);
-      QueryResultBatch batch = response.getBatch();
-      if (batch.getEntityResultCount() == 0) {
-        throw new NoSuchElementException(
-            "Datastore total statistics for dataset " + datasetId + " unavailable");
-      }
-      Entity entity = batch.getEntityResult(0).getEntity();
-      return getPropertyMap(entity).get("timestamp").getTimestampMicrosecondsValue();
-    }
-
-    private Datastore getDatastore(PipelineOptions pipelineOptions) {
-      DatastoreOptions.Builder builder =
-          new DatastoreOptions.Builder().host(host).dataset(datasetId).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(host, datasetId, query, namespace);
-      res.mockSplitter = splitter;
-      res.mockEstimateSizeBytes = mockEstimateSizeBytes;
-      return res;
-    }
-
-    /** For testing only. */
-    Source withMockEstimateSizeBytes(Long estimateSizeBytes) {
-      Source res = new Source(host, datasetId, query, namespace);
-      res.mockSplitter = mockSplitter;
-      res.mockEstimateSizeBytes = estimateSizeBytes;
-      return res;
-    }
-  }
-
-  ///////////////////// Write Class /////////////////////////////////
-
-  /**
-   * Returns a new {@link DatastoreIO.Sink} builder using the default host.
-   * You need to further configure it using {@link DatastoreIO.Sink#withDataset}, and optionally
-   * {@link DatastoreIO.Sink#withHost} before using it in a {@link Write} transform.
-   *
-   * <p>For example: {@code p.apply(Write.to(DatastoreIO.sink().withDataset(dataset)));}
-   */
-  public static Sink sink() {
-    return new Sink(DEFAULT_HOST, null);
-  }
-
-  /**
-   * Returns a new {@link Write} transform that will write to a {@link Sink}.
-   *
-   * <p>For example: {@code p.apply(DatastoreIO.writeTo(dataset));}
-   */
-  public static Write.Bound<Entity> writeTo(String datasetId) {
-    return Write.to(sink().withDataset(datasetId));
-  }
-
-  /**
-   * A {@link Sink} that writes a {@link PCollection} containing
-   * {@link Entity Entities} to a Datastore kind.
-   *
-   */
-  public static class Sink extends com.google.cloud.dataflow.sdk.io.Sink<Entity> {
-    final String host;
-    final String datasetId;
-
-    /**
-     * Returns a {@link Sink} that is like this one, but will write to the specified dataset.
-     */
-    public Sink withDataset(String datasetId) {
-      checkNotNull(datasetId, "datasetId");
-      return new Sink(host, datasetId);
-    }
-
-    /**
-     * Returns a {@link Sink} that is like this one, but will use the given host.  If not specified,
-     * the {@link DatastoreIO#DEFAULT_HOST default host} will be used.
-     */
-    public Sink withHost(String host) {
-      checkNotNull(host, "host");
-      return new Sink(host, datasetId);
-    }
-
-    /**
-     * Constructs a Sink with given host and dataset.
-     */
-    protected Sink(String host, String datasetId) {
-      this.host = checkNotNull(host, "host");
-      this.datasetId = datasetId;
-    }
-
-    /**
-     * Ensures the host and dataset are set.
-     */
-    @Override
-    public void validate(PipelineOptions options) {
-      Preconditions.checkNotNull(
-          host, "Host is a required parameter. Please use withHost to set the host.");
-      Preconditions.checkNotNull(
-          datasetId,
-          "Dataset ID is a required parameter. Please use withDataset to to set the datasetId.");
-    }
-
-    @Override
-    public DatastoreWriteOperation createWriteOperation(PipelineOptions options) {
-      return new DatastoreWriteOperation(this);
-    }
-  }
-
-  /**
-   * 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()
-              .host(sink.host)
-              .dataset(sink.datasetId)
-              .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.getPathElementList();
-      if (elementList.isEmpty()) {
-        return false;
-      }
-      PathElement lastElement = elementList.get(elementList.size() - 1);
-      return (lastElement.hasId() || lastElement.hasName());
-    }
-
-    // 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();
-          commitRequest.getMutationBuilder().addAllUpsert(entities);
-          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 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() : Integer.MAX_VALUE;
-    }
-
-    @Override
-    public Entity getCurrent() {
-      return currentEntity;
-    }
-
-    @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;
-        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(Math.min(userLimit, QUERY_BATCH_LIMIT));
-      if (currentBatch != null && currentBatch.hasEndCursor()) {
-        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.getEntityResultCount();
-      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.getEntityResultList().iterator();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java
deleted file mode 100644
index f30960c..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java
+++ /dev/null
@@ -1,865 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.api.client.googleapis.batch.BatchRequest;
-import com.google.api.client.googleapis.batch.json.JsonBatchCallback;
-import com.google.api.client.googleapis.json.GoogleJsonError;
-import com.google.api.client.http.HttpHeaders;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.services.storage.Storage;
-import com.google.api.services.storage.StorageRequest;
-import com.google.api.services.storage.model.StorageObject;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.FileIOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.base.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.channels.WritableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-import javax.annotation.concurrent.NotThreadSafe;
-
-/**
- * Abstract {@link Sink} for file-based output. An implementation of FileBasedSink writes file-based
- * output and defines the format of output files (how values are written, headers/footers, MIME
- * type, etc.).
- *
- * <p>At pipeline construction time, the methods of FileBasedSink are called to validate the sink
- * and to create a {@link Sink.WriteOperation} that manages the process of writing to the sink.
- *
- * <p>The process of writing to file-based sink is as follows:
- * <ol>
- * <li>An optional subclass-defined initialization,
- * <li>a parallel write of bundles to temporary files, and finally,
- * <li>these temporary files are renamed with final output filenames.
- * </ol>
- *
- * <p>Supported file systems are those registered with {@link IOChannelUtils}.
- *
- * @param <T> the type of values written to the sink.
- */
-public abstract class FileBasedSink<T> extends Sink<T> {
-  /**
-   * Base filename for final output files.
-   */
-  protected final String baseOutputFilename;
-
-  /**
-   * The extension to be used for the final output files.
-   */
-  protected final String extension;
-
-  /**
-   * Naming template for output files. See {@link ShardNameTemplate} for a description of
-   * possible naming templates.  Default is {@link ShardNameTemplate#INDEX_OF_MAX}.
-   */
-  protected final String fileNamingTemplate;
-
-  /**
-   * Construct a FileBasedSink with the given base output filename and extension.
-   */
-  public FileBasedSink(String baseOutputFilename, String extension) {
-    this(baseOutputFilename, extension, ShardNameTemplate.INDEX_OF_MAX);
-  }
-
-  /**
-   * Construct a FileBasedSink with the given base output filename, extension, and file naming
-   * template.
-   *
-   * <p>See {@link ShardNameTemplate} for a description of file naming templates.
-   */
-  public FileBasedSink(String baseOutputFilename, String extension, String fileNamingTemplate) {
-    this.baseOutputFilename = baseOutputFilename;
-    this.extension = extension;
-    this.fileNamingTemplate = fileNamingTemplate;
-  }
-
-  /**
-   * Returns the base output filename for this file based sink.
-   */
-  public String getBaseOutputFilename() {
-    return baseOutputFilename;
-  }
-
-  /**
-   * 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.
-   */
-  @Override
-  public void validate(PipelineOptions options) {}
-
-  /**
-   * Return a subclass of {@link FileBasedSink.FileBasedWriteOperation} that will manage the write
-   * to the sink.
-   */
-  @Override
-  public abstract FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options);
-
-  /**
-   * Abstract {@link Sink.WriteOperation} that manages the process of writing to a
-   * {@link FileBasedSink}.
-   *
-   * <p>The primary responsibilities of the FileBasedWriteOperation is the management of output
-   * files. During a write, {@link FileBasedSink.FileBasedWriter}s write bundles to temporary file
-   * locations. After the bundles have been written,
-   * <ol>
-   * <li>{@link FileBasedSink.FileBasedWriteOperation#finalize} is given a list of the temporary
-   * files containing the output bundles.
-   * <li>During finalize, these temporary files are copied to final output locations and named
-   * according to a file naming template.
-   * <li>Finally, any temporary files that were created during the write are removed.
-   * </ol>
-   *
-   * <p>Subclass implementations of FileBasedWriteOperation must implement
-   * {@link FileBasedSink.FileBasedWriteOperation#createWriter} to return a concrete
-   * FileBasedSinkWriter.
-   *
-   * <h2>Temporary and Output File Naming:</h2> During the write, bundles are written to temporary
-   * files using the baseTemporaryFilename that can be provided via the constructor of
-   * FileBasedWriteOperation. These temporary files will be named
-   * {@code {baseTemporaryFilename}-temp-{bundleId}}, where bundleId is the unique id of the bundle.
-   * For example, if baseTemporaryFilename is "gs://my-bucket/my_temp_output", the output for a
-   * bundle with bundle id 15723 will be "gs://my-bucket/my_temp_output-temp-15723".
-   *
-   * <p>Final output files are written to baseOutputFilename with the format
-   * {@code {baseOutputFilename}-0000i-of-0000n.{extension}} where n is the total number of bundles
-   * written and extension is the file extension. Both baseOutputFilename and extension are required
-   * constructor arguments.
-   *
-   * <p>Subclass implementations can change the file naming template by supplying a value for
-   * {@link FileBasedSink#fileNamingTemplate}.
-   *
-   * <h2>Temporary Bundle File Handling:</h2>
-   * <p>{@link FileBasedSink.FileBasedWriteOperation#temporaryFileRetention} controls the behavior
-   * for managing temporary files. By default, temporary files will be removed. Subclasses can
-   * provide a different value to the constructor.
-   *
-   * <p>Note that in the case of permanent failure of a bundle's write, no clean up of temporary
-   * files will occur.
-   *
-   * <p>If there are no elements in the PCollection being written, no output will be generated.
-   *
-   * @param <T> the type of values written to the sink.
-   */
-  public abstract static class FileBasedWriteOperation<T> extends WriteOperation<T, FileResult> {
-    private static final Logger LOG = LoggerFactory.getLogger(FileBasedWriteOperation.class);
-
-    /**
-     * Options for handling of temporary output files.
-     */
-    public enum TemporaryFileRetention {
-      KEEP,
-      REMOVE;
-    }
-
-    /**
-     * The Sink that this WriteOperation will write to.
-     */
-    protected final FileBasedSink<T> sink;
-
-    /**
-     * Option to keep or remove temporary output files.
-     */
-    protected final TemporaryFileRetention temporaryFileRetention;
-
-    /**
-     * Base filename used for temporary output files. Default is the baseOutputFilename.
-     */
-    protected final String baseTemporaryFilename;
-
-    /**
-     * Name separator for temporary files. Temporary files will be named
-     * {@code {baseTemporaryFilename}-temp-{bundleId}}.
-     */
-    protected static final String TEMPORARY_FILENAME_SEPARATOR = "-temp-";
-
-    /**
-     * Build a temporary filename using the temporary filename separator with the given prefix and
-     * suffix.
-     */
-    protected static final String buildTemporaryFilename(String prefix, String suffix) {
-      return prefix + FileBasedWriteOperation.TEMPORARY_FILENAME_SEPARATOR + suffix;
-    }
-
-    /**
-     * Construct a FileBasedWriteOperation using the same base filename for both temporary and
-     * output files.
-     *
-     * @param sink the FileBasedSink that will be used to configure this write operation.
-     */
-    public FileBasedWriteOperation(FileBasedSink<T> sink) {
-      this(sink, sink.baseOutputFilename);
-    }
-
-    /**
-     * Construct a FileBasedWriteOperation.
-     *
-     * @param sink the FileBasedSink that will be used to configure this write operation.
-     * @param baseTemporaryFilename the base filename to be used for temporary output files.
-     */
-    public FileBasedWriteOperation(FileBasedSink<T> sink, String baseTemporaryFilename) {
-      this(sink, baseTemporaryFilename, TemporaryFileRetention.REMOVE);
-    }
-
-    /**
-     * Create a new FileBasedWriteOperation.
-     *
-     * @param sink the FileBasedSink that will be used to configure this write operation.
-     * @param baseTemporaryFilename the base filename to be used for temporary output files.
-     * @param temporaryFileRetention defines how temporary files are handled.
-     */
-    public FileBasedWriteOperation(FileBasedSink<T> sink, String baseTemporaryFilename,
-        TemporaryFileRetention temporaryFileRetention) {
-      this.sink = sink;
-      this.baseTemporaryFilename = baseTemporaryFilename;
-      this.temporaryFileRetention = temporaryFileRetention;
-    }
-
-    /**
-     * Clients must implement to return a subclass of {@link FileBasedSink.FileBasedWriter}. This
-     * method must satisfy the restrictions placed on implementations of
-     * {@link Sink.WriteOperation#createWriter}. Namely, it must not mutate the state of the object.
-     */
-    @Override
-    public abstract FileBasedWriter<T> createWriter(PipelineOptions options) throws Exception;
-
-    /**
-     * Initialization of the sink. Default implementation is a no-op. May be overridden by subclass
-     * implementations to perform initialization of the sink at pipeline runtime. This method must
-     * be idempotent and is subject to the same implementation restrictions as
-     * {@link Sink.WriteOperation#initialize}.
-     */
-    @Override
-    public void initialize(PipelineOptions options) throws Exception {}
-
-    /**
-     * Finalizes writing by copying temporary output files to their final location and optionally
-     * removing temporary files.
-     *
-     * <p>Finalization may be overridden by subclass implementations to perform customized
-     * finalization (e.g., initiating some operation on output bundles, merging them, etc.).
-     * {@code writerResults} contains the filenames of written bundles.
-     *
-     * <p>If subclasses override this method, they must guarantee that its implementation is
-     * idempotent, as it may be executed multiple times in the case of failure or for redundancy. It
-     * is a best practice to attempt to try to make this method atomic.
-     *
-     * @param writerResults the results of writes (FileResult).
-     */
-    @Override
-    public void finalize(Iterable<FileResult> writerResults, PipelineOptions options)
-        throws Exception {
-      // Collect names of temporary files and rename them.
-      List<String> files = new ArrayList<>();
-      for (FileResult result : writerResults) {
-        LOG.debug("Temporary bundle output file {} will be copied.", result.getFilename());
-        files.add(result.getFilename());
-      }
-      copyToOutputFiles(files, options);
-
-      // Optionally remove temporary files.
-      if (temporaryFileRetention == TemporaryFileRetention.REMOVE) {
-        removeTemporaryFiles(options);
-      }
-    }
-
-    /**
-     * Copy temporary files to final output filenames using the file naming template.
-     *
-     * <p>Can be called from subclasses that override {@link FileBasedWriteOperation#finalize}.
-     *
-     * <p>Files will be named according to the file naming template. The order of the output files
-     * will be the same as the sorted order of the input filenames.  In other words, if the input
-     * filenames are ["C", "A", "B"], baseOutputFilename is "file", the extension is ".txt", and
-     * the fileNamingTemplate is "-SSS-of-NNN", the contents of A will be copied to
-     * file-000-of-003.txt, the contents of B will be copied to file-001-of-003.txt, etc.
-     *
-     * @param filenames the filenames of temporary files.
-     * @return a list containing the names of final output files.
-     */
-    protected final List<String> copyToOutputFiles(List<String> filenames, PipelineOptions options)
-        throws IOException {
-      int numFiles = filenames.size();
-      List<String> srcFilenames = new ArrayList<>();
-      List<String> destFilenames = generateDestinationFilenames(numFiles);
-
-      // Sort files for copying.
-      srcFilenames.addAll(filenames);
-      Collections.sort(srcFilenames);
-
-      if (numFiles > 0) {
-        LOG.debug("Copying {} files.", numFiles);
-        FileOperations fileOperations =
-            FileOperationsFactory.getFileOperations(destFilenames.get(0), options);
-        fileOperations.copy(srcFilenames, destFilenames);
-      } else {
-        LOG.info("No output files to write.");
-      }
-
-      return destFilenames;
-    }
-
-    /**
-     * Generate output bundle filenames.
-     */
-    protected final List<String> generateDestinationFilenames(int numFiles) {
-      List<String> destFilenames = new ArrayList<>();
-      String extension = getSink().extension;
-      String baseOutputFilename = getSink().baseOutputFilename;
-      String fileNamingTemplate = getSink().fileNamingTemplate;
-
-      String suffix = getFileExtension(extension);
-      for (int i = 0; i < numFiles; i++) {
-        destFilenames.add(IOChannelUtils.constructName(
-            baseOutputFilename, fileNamingTemplate, suffix, i, numFiles));
-      }
-      return destFilenames;
-    }
-
-    /**
-     * Returns the file extension to be used. If the user did not request a file
-     * extension then this method returns the empty string. Otherwise this method
-     * adds a {@code "."} to the beginning of the users extension if one is not present.
-     */
-    private String getFileExtension(String usersExtension) {
-      if (usersExtension == null || usersExtension.isEmpty()) {
-        return "";
-      }
-      if (usersExtension.startsWith(".")) {
-        return usersExtension;
-      }
-      return "." + usersExtension;
-    }
-
-    /**
-     * Removes temporary output files. Uses the temporary filename to find files to remove.
-     *
-     * <p>Can be called from subclasses that override {@link FileBasedWriteOperation#finalize}.
-     * <b>Note:</b>If finalize is overridden and does <b>not</b> rename or otherwise finalize
-     * temporary files, this method will remove them.
-     */
-    protected final void removeTemporaryFiles(PipelineOptions options) throws IOException {
-      String pattern = buildTemporaryFilename(baseTemporaryFilename, "*");
-      LOG.debug("Finding temporary bundle output files matching {}.", pattern);
-      FileOperations fileOperations = FileOperationsFactory.getFileOperations(pattern, options);
-      IOChannelFactory factory = IOChannelUtils.getFactory(pattern);
-      Collection<String> matches = factory.match(pattern);
-      LOG.debug("{} temporary files matched {}", matches.size(), pattern);
-      LOG.debug("Removing {} files.", matches.size());
-      fileOperations.remove(matches);
-    }
-
-    /**
-     * Provides a coder for {@link FileBasedSink.FileResult}.
-     */
-    @Override
-    public Coder<FileResult> getWriterResultCoder() {
-      return SerializableCoder.of(FileResult.class);
-    }
-
-    /**
-     * Returns the FileBasedSink for this write operation.
-     */
-    @Override
-    public FileBasedSink<T> getSink() {
-      return sink;
-    }
-  }
-
-  /**
-   * Abstract {@link Sink.Writer} that writes a bundle to a {@link FileBasedSink}. Subclass
-   * implementations provide a method that can write a single value to a {@link WritableByteChannel}
-   * ({@link Sink.Writer#write}).
-   *
-   * <p>Subclass implementations may also override methods that write headers and footers before and
-   * after the values in a bundle, respectively, as well as provide a MIME type for the output
-   * channel.
-   *
-   * <p>Multiple FileBasedWriter instances may be created on the same worker, and therefore any
-   * access to static members or methods should be thread safe.
-   *
-   * @param <T> the type of values to write.
-   */
-  public abstract static class FileBasedWriter<T> extends Writer<T, FileResult> {
-    private static final Logger LOG = LoggerFactory.getLogger(FileBasedWriter.class);
-
-    final FileBasedWriteOperation<T> writeOperation;
-
-    /**
-     * Unique id for this output bundle.
-     */
-    private String id;
-
-    /**
-     * The filename of the output bundle. Equal to the
-     * {@link FileBasedSink.FileBasedWriteOperation#TEMPORARY_FILENAME_SEPARATOR} and id appended to
-     * the baseName.
-     */
-    private String filename;
-
-    /**
-     * The channel to write to.
-     */
-    private WritableByteChannel channel;
-
-    /**
-     * The MIME type used in the creation of the output channel (if the file system supports it).
-     *
-     * <p>GCS, for example, supports writing files with Content-Type metadata.
-     *
-     * <p>May be overridden. Default is {@link MimeTypes#TEXT}. See {@link MimeTypes} for other
-     * options.
-     */
-    protected String mimeType = MimeTypes.TEXT;
-
-    /**
-     * Construct a new FileBasedWriter with a base filename.
-     */
-    public FileBasedWriter(FileBasedWriteOperation<T> writeOperation) {
-      Preconditions.checkNotNull(writeOperation);
-      this.writeOperation = writeOperation;
-    }
-
-    /**
-     * Called with the channel that a subclass will write its header, footer, and values to.
-     * Subclasses should either keep a reference to the channel provided or create and keep a
-     * reference to an appropriate object that they will use to write to it.
-     *
-     * <p>Called before any subsequent calls to writeHeader, writeFooter, and write.
-     */
-    protected abstract void prepareWrite(WritableByteChannel channel) throws Exception;
-
-    /**
-     * Writes header at the beginning of output files. Nothing by default; subclasses may override.
-     */
-    protected void writeHeader() throws Exception {}
-
-    /**
-     * Writes footer at the end of output files. Nothing by default; subclasses may override.
-     */
-    protected void writeFooter() throws Exception {}
-
-    /**
-     * Opens the channel.
-     */
-    @Override
-    public final void open(String uId) throws Exception {
-      this.id = uId;
-      filename = FileBasedWriteOperation.buildTemporaryFilename(
-          getWriteOperation().baseTemporaryFilename, uId);
-      LOG.debug("Opening {}.", filename);
-      channel = IOChannelUtils.create(filename, mimeType);
-      try {
-        prepareWrite(channel);
-        LOG.debug("Writing header to {}.", filename);
-        writeHeader();
-      } catch (Exception e) {
-        // The caller shouldn't have to close() this Writer if it fails to open(), so close the
-        // channel if prepareWrite() or writeHeader() fails.
-        try {
-          LOG.error("Writing header to {} failed, closing channel.", filename);
-          channel.close();
-        } catch (IOException closeException) {
-          // Log exception and mask it.
-          LOG.error("Closing channel for {} failed: {}", filename, closeException.getMessage());
-        }
-        // Throw the exception that caused the write to fail.
-        throw e;
-      }
-      LOG.debug("Starting write of bundle {} to {}.", this.id, filename);
-    }
-
-    /**
-     * Closes the channel and return the bundle result.
-     */
-    @Override
-    public final FileResult close() throws Exception {
-      try (WritableByteChannel theChannel = channel) {
-        LOG.debug("Writing footer to {}.", filename);
-        writeFooter();
-      }
-      FileResult result = new FileResult(filename);
-      LOG.debug("Result for bundle {}: {}", this.id, filename);
-      return result;
-    }
-
-    /**
-     * Return the FileBasedWriteOperation that this Writer belongs to.
-     */
-    @Override
-    public FileBasedWriteOperation<T> getWriteOperation() {
-      return writeOperation;
-    }
-  }
-
-  /**
-   * Result of a single bundle write. Contains the filename of the bundle.
-   */
-  public static final class FileResult implements Serializable {
-    private final String filename;
-
-    public FileResult(String filename) {
-      this.filename = filename;
-    }
-
-    public String getFilename() {
-      return filename;
-    }
-  }
-
-  // File system operations
-  // Warning: These class are purposefully private and will be replaced by more robust file I/O
-  // utilities. Not for use outside FileBasedSink.
-
-  /**
-   * Factory for FileOperations.
-   */
-  private static class FileOperationsFactory {
-    /**
-     * Return a FileOperations implementation based on which IOChannel would be used to write to a
-     * location specification (not necessarily a filename, as it may contain wildcards).
-     *
-     * <p>Only supports File and GCS locations (currently, the only factories registered with
-     * IOChannelUtils). For other locations, an exception is thrown.
-     */
-    public static FileOperations getFileOperations(String spec, PipelineOptions options)
-        throws IOException {
-      IOChannelFactory factory = IOChannelUtils.getFactory(spec);
-      if (factory instanceof GcsIOChannelFactory) {
-        return new GcsOperations(options);
-      } else if (factory instanceof FileIOChannelFactory) {
-        return new LocalFileOperations();
-      } else {
-        throw new IOException("Unrecognized file system.");
-      }
-    }
-  }
-
-  /**
-   * Copy and Remove operations for files. Operations behave like remove-if-existing and
-   * copy-if-existing and do not throw exceptions on file not found to enable retries of these
-   * operations in the case of transient error.
-   */
-  private static interface FileOperations {
-    /**
-     * Copy a collection of files from one location to another.
-     *
-     * <p>The number of source filenames must equal the number of destination filenames.
-     *
-     * @param srcFilenames the source filenames.
-     * @param destFilenames the destination filenames.
-     */
-    public void copy(List<String> srcFilenames, List<String> destFilenames) throws IOException;
-
-    /**
-     * Remove a collection of files.
-     */
-    public void remove(Collection<String> filenames) throws IOException;
-  }
-
-  /**
-   * GCS file system operations.
-   */
-  private static class GcsOperations implements FileOperations {
-    private static final Logger LOG = LoggerFactory.getLogger(GcsOperations.class);
-
-    /**
-     * Maximum number of requests permitted in a GCS batch request.
-     */
-    private static final int MAX_REQUESTS_PER_BATCH = 1000;
-
-    private ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-    private GcsOptions gcsOptions;
-    private Storage gcs;
-    private BatchHelper batchHelper;
-
-    public GcsOperations(PipelineOptions options) {
-      gcsOptions = options.as(GcsOptions.class);
-      gcs = Transport.newStorageClient(gcsOptions).build();
-      batchHelper =
-          new BatchHelper(gcs.getRequestFactory().getInitializer(), gcs, MAX_REQUESTS_PER_BATCH);
-    }
-
-    @Override
-    public void copy(List<String> srcFilenames, List<String> destFilenames) throws IOException {
-      Preconditions.checkArgument(
-          srcFilenames.size() == destFilenames.size(),
-          String.format("Number of source files {} must equal number of destination files {}",
-              srcFilenames.size(), destFilenames.size()));
-      for (int i = 0; i < srcFilenames.size(); i++) {
-        final GcsPath sourcePath = GcsPath.fromUri(srcFilenames.get(i));
-        final GcsPath destPath = GcsPath.fromUri(destFilenames.get(i));
-        LOG.debug("Copying {} to {}", sourcePath, destPath);
-        Storage.Objects.Copy copyObject = gcs.objects().copy(sourcePath.getBucket(),
-            sourcePath.getObject(), destPath.getBucket(), destPath.getObject(), null);
-        batchHelper.queue(copyObject, new JsonBatchCallback<StorageObject>() {
-          @Override
-          public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) {
-            LOG.debug("Successfully copied {} to {}", sourcePath, destPath);
-          }
-
-          @Override
-          public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
-            // Do nothing on item not found.
-            if (!errorExtractor.itemNotFound(e)) {
-              throw new IOException(e.toString());
-            }
-            LOG.debug("{} does not exist.", sourcePath);
-          }
-        });
-      }
-      batchHelper.flush();
-    }
-
-    @Override
-    public void remove(Collection<String> filenames) throws IOException {
-      for (String filename : filenames) {
-        final GcsPath path = GcsPath.fromUri(filename);
-        LOG.debug("Removing: " + path);
-        Storage.Objects.Delete deleteObject =
-            gcs.objects().delete(path.getBucket(), path.getObject());
-        batchHelper.queue(deleteObject, new JsonBatchCallback<Void>() {
-          @Override
-          public void onSuccess(Void obj, HttpHeaders responseHeaders) throws IOException {
-            LOG.debug("Successfully removed {}", path);
-          }
-
-          @Override
-          public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException {
-            // Do nothing on item not found.
-            if (!errorExtractor.itemNotFound(e)) {
-              throw new IOException(e.toString());
-            }
-            LOG.debug("{} does not exist.", path);
-          }
-        });
-      }
-      batchHelper.flush();
-    }
-  }
-
-  /**
-   * File systems supported by {@link Files}.
-   */
-  private static class LocalFileOperations implements FileOperations {
-    private static final Logger LOG = LoggerFactory.getLogger(LocalFileOperations.class);
-
-    @Override
-    public void copy(List<String> srcFilenames, List<String> destFilenames) throws IOException {
-      Preconditions.checkArgument(
-          srcFilenames.size() == destFilenames.size(),
-          String.format("Number of source files {} must equal number of destination files {}",
-              srcFilenames.size(), destFilenames.size()));
-      int numFiles = srcFilenames.size();
-      for (int i = 0; i < numFiles; i++) {
-        String src = srcFilenames.get(i);
-        String dst = destFilenames.get(i);
-        LOG.debug("Copying {} to {}", src, dst);
-        copyOne(src, dst);
-      }
-    }
-
-    private void copyOne(String source, String destination) throws IOException {
-      try {
-        // Copy the source file, replacing the existing destination.
-        Files.copy(Paths.get(source), Paths.get(destination), StandardCopyOption.REPLACE_EXISTING);
-      } catch (NoSuchFileException e) {
-        LOG.debug("{} does not exist.", source);
-        // Suppress exception if file does not exist.
-      }
-    }
-
-    @Override
-    public void remove(Collection<String> filenames) throws IOException {
-      for (String filename : filenames) {
-        LOG.debug("Removing file {}", filename);
-        removeOne(filename);
-      }
-    }
-
-    private void removeOne(String filename) throws IOException {
-      // Delete the file if it exists.
-      boolean exists = Files.deleteIfExists(Paths.get(filename));
-      if (!exists) {
-        LOG.debug("{} does not exist.", filename);
-      }
-    }
-  }
-
-  /**
-   * BatchHelper abstracts out the logic for the maximum requests per batch for GCS.
-   *
-   * <p>Copy of
-   * https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/src/main/java/com/google/cloud/hadoop/gcsio/BatchHelper.java
-   *
-   * <p>Copied to prevent Dataflow from depending on the Hadoop-related dependencies that are not
-   * used in Dataflow.  Hadoop-related dependencies will be removed from the Google Cloud Storage
-   * Connector (https://cloud.google.com/hadoop/google-cloud-storage-connector) so that this project
-   * and others may use the connector without introducing unnecessary dependencies.
-   *
-   * <p>This class is not thread-safe; create a new BatchHelper instance per single-threaded logical
-   * grouping of requests.
-   */
-  @NotThreadSafe
-  private static class BatchHelper {
-    /**
-     * Callback that causes a single StorageRequest to be added to the BatchRequest.
-     */
-    protected static interface QueueRequestCallback {
-      void enqueue() throws IOException;
-    }
-
-    private final List<QueueRequestCallback> pendingBatchEntries;
-    private final BatchRequest batch;
-
-    // Number of requests that can be queued into a single actual HTTP request
-    // before a sub-batch is sent.
-    private final long maxRequestsPerBatch;
-
-    // Flag that indicates whether there is an in-progress flush.
-    private boolean flushing = false;
-
-    /**
-     * Primary constructor, generally accessed only via the inner Factory class.
-     */
-    public BatchHelper(
-        HttpRequestInitializer requestInitializer, Storage gcs, long maxRequestsPerBatch) {
-      this.pendingBatchEntries = new LinkedList<>();
-      this.batch = gcs.batch(requestInitializer);
-      this.maxRequestsPerBatch = maxRequestsPerBatch;
-    }
-
-    /**
-     * Adds an additional request to the batch, and possibly flushes the current contents of the
-     * batch if {@code maxRequestsPerBatch} has been reached.
-     */
-    public <T> void queue(final StorageRequest<T> req, final JsonBatchCallback<T> callback)
-        throws IOException {
-      QueueRequestCallback queueCallback = new QueueRequestCallback() {
-        @Override
-        public void enqueue() throws IOException {
-          req.queue(batch, callback);
-        }
-      };
-      pendingBatchEntries.add(queueCallback);
-
-      flushIfPossibleAndRequired();
-    }
-
-    // Flush our buffer if we have more pending entries than maxRequestsPerBatch
-    private void flushIfPossibleAndRequired() throws IOException {
-      if (pendingBatchEntries.size() > maxRequestsPerBatch) {
-        flushIfPossible();
-      }
-    }
-
-    // Flush our buffer if we are not already in a flush operation and we have data to flush.
-    private void flushIfPossible() throws IOException {
-      if (!flushing && pendingBatchEntries.size() > 0) {
-        flushing = true;
-        try {
-          while (batch.size() < maxRequestsPerBatch && pendingBatchEntries.size() > 0) {
-            QueueRequestCallback head = pendingBatchEntries.remove(0);
-            head.enqueue();
-          }
-
-          batch.execute();
-        } finally {
-          flushing = false;
-        }
-      }
-    }
-
-
-    /**
-     * Sends any currently remaining requests in the batch; should be called at the end of any
-     * series of batched requests to ensure everything has been sent.
-     */
-    public void flush() throws IOException {
-      flushIfPossible();
-    }
-  }
-
-  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);
-                  }
-                }
-              }));
-    }
-  }
-}


[38/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
new file mode 100644
index 0000000..b907ae7
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.complete.game.UserScore.ExtractAndSumScore;
+import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
+import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tests of UserScore.
+ */
+@RunWith(JUnit4.class)
+public class UserScoreTest implements Serializable {
+
+  static final String[] GAME_EVENTS_ARRAY = new String[] {
+    "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444",
+    "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444",
+    "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444",
+    "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444",
+    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444",
+    "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444",
+    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444",
+    "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444",
+    "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444",
+    "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444"
+  };
+
+    static final String[] GAME_EVENTS_ARRAY2 = new String[] {
+    "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444",
+    "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444",
+    "user13_BisqueBilby,BisqueBilby,xxx,1447955630000,2015-11-19 09:53:53.444"
+  };
+
+  static final List<String> GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY);
+  static final List<String> GAME_EVENTS2 = Arrays.asList(GAME_EVENTS_ARRAY2);
+
+  static final List<KV<String, Integer>> USER_SUMS = Arrays.asList(
+      KV.of("user0_MagentaKangaroo", 3), KV.of("user13_ApricotQuokka", 15),
+      KV.of("user6_AmberNumbat", 11), KV.of("user7_AlmondWallaby", 15),
+      KV.of("user7_AndroidGreenKookaburra", 23),
+      KV.of("user19_BisqueBilby", 14));
+
+  static final List<KV<String, Integer>> TEAM_SUMS = Arrays.asList(
+      KV.of("MagentaKangaroo", 3), KV.of("ApricotQuokka", 15),
+      KV.of("AmberNumbat", 11), KV.of("AlmondWallaby", 15),
+      KV.of("AndroidGreenKookaburra", 23),
+      KV.of("BisqueBilby", 14));
+
+  /** Test the ParseEventFn DoFn. */
+  @Test
+  public void testParseEventFn() {
+    DoFnTester<String, GameActionInfo> parseEventFn =
+        DoFnTester.of(new ParseEventFn());
+
+    List<GameActionInfo> results = parseEventFn.processBatch(GAME_EVENTS_ARRAY);
+    Assert.assertEquals(results.size(), 8);
+    Assert.assertEquals(results.get(0).getUser(), "user0_MagentaKangaroo");
+    Assert.assertEquals(results.get(0).getTeam(), "MagentaKangaroo");
+    Assert.assertEquals(results.get(0).getScore(), new Integer(3));
+  }
+
+  /** Tests ExtractAndSumScore("user"). */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUserScoreSums() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
+
+    PCollection<KV<String, Integer>> output = input
+      .apply(ParDo.of(new ParseEventFn()))
+      // Extract and sum username/score pairs from the event data.
+      .apply("ExtractUserScore", new ExtractAndSumScore("user"));
+
+    // Check the user score sums.
+    PAssert.that(output).containsInAnyOrder(USER_SUMS);
+
+    p.run();
+  }
+
+  /** Tests ExtractAndSumScore("team"). */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testTeamScoreSums() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
+
+    PCollection<KV<String, Integer>> output = input
+      .apply(ParDo.of(new ParseEventFn()))
+      // Extract and sum teamname/score pairs from the event data.
+      .apply("ExtractTeamScore", new ExtractAndSumScore("team"));
+
+    // Check the team score sums.
+    PAssert.that(output).containsInAnyOrder(TEAM_SUMS);
+
+    p.run();
+  }
+
+  /** Test that bad input data is dropped appropriately. */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUserScoresBadInput() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(GAME_EVENTS2).withCoder(StringUtf8Coder.of()));
+
+    PCollection<KV<String, Integer>> extract = input
+      .apply(ParDo.of(new ParseEventFn()))
+      .apply(
+          MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
+          .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
+
+    PAssert.that(extract).empty();
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java
deleted file mode 100644
index 6bbafdd..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java
+++ /dev/null
@@ -1,50 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.io.PrintStream;
-
-/**
- * Options that are used to configure the {@link BlockingDataflowPipelineRunner}.
- */
-@Description("Configure options on the BlockingDataflowPipelineRunner.")
-public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions {
-  /**
-   * Output stream for job status messages.
-   */
-  @Description("Where messages generated during execution of the Dataflow job will be output.")
-  @JsonIgnore
-  @Hidden
-  @Default.InstanceFactory(StandardOutputFactory.class)
-  PrintStream getJobMessageOutput();
-  void setJobMessageOutput(PrintStream value);
-
-  /**
-   * Returns a default of {@link System#out}.
-   */
-  public static class StandardOutputFactory implements DefaultValueFactory<PrintStream> {
-    @Override
-    public PrintStream create(PipelineOptions options) {
-      return System.out;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java
deleted file mode 100644
index 3f0503e..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java
+++ /dev/null
@@ -1,53 +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 com.google.cloud.dataflow.sdk.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.clouddebugger.v2.model.Debuggee;
-
-import javax.annotation.Nullable;
-
-/**
- * Options for controlling Cloud Debugger.
- */
-@Description("[Experimental] Used to configure the Cloud Debugger")
-@Experimental
-@Hidden
-public interface CloudDebuggerOptions {
-
-  /** Whether to enable the Cloud Debugger snapshot agent for the current job. */
-  @Description("Whether to enable the Cloud Debugger snapshot agent for the current job.")
-  boolean getEnableCloudDebugger();
-  void setEnableCloudDebugger(boolean enabled);
-
-  /** The Cloud Debugger debuggee to associate with. This should not be set directly. */
-  @Description("The Cloud Debugger debuggee to associate with. This should not be set directly.")
-  @Hidden
-  @Nullable Debuggee getDebuggee();
-  void setDebuggee(Debuggee debuggee);
-
-  /** The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. */
-  @Description(
-      "The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. "
-      + "Should be a double between 0 and 1. "
-      + "Snapshots will be cancelled if evaluating conditions takes more than this ratio of time.")
-  @Default.Double(0.01)
-  double getMaxConditionCost();
-  void setMaxConditionCost(double maxConditionCost);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java
deleted file mode 100644
index 1be93eb..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java
+++ /dev/null
@@ -1,254 +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 com.google.cloud.dataflow.sdk.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.cloud.dataflow.sdk.util.DataflowPathValidator;
-import com.google.cloud.dataflow.sdk.util.DataflowTransport;
-import com.google.cloud.dataflow.sdk.util.GcsStager;
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.util.PathValidator;
-import com.google.cloud.dataflow.sdk.util.Stager;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * Internal. Options used to control execution of the Dataflow SDK for
- * debugging and testing purposes.
- */
-@Description("[Internal] Options used to control execution of the Dataflow SDK for "
-    + "debugging and testing purposes.")
-@Hidden
-public interface DataflowPipelineDebugOptions extends PipelineOptions {
-
-  /**
-   * The list of backend experiments to enable.
-   *
-   * <p>Dataflow provides a number of experimental features that can be enabled
-   * with this flag.
-   *
-   * <p>Please sync with the Dataflow team before enabling any experiments.
-   */
-  @Description("[Experimental] Dataflow provides a number of experimental features that can "
-      + "be enabled with this flag. Please sync with the Dataflow team before enabling any "
-      + "experiments.")
-  @Experimental
-  List<String> getExperiments();
-  void setExperiments(List<String> value);
-
-  /**
-   * The root URL for the Dataflow API. {@code dataflowEndpoint} can override this value
-   * if it contains an absolute URL, otherwise {@code apiRootUrl} will be combined with
-   * {@code dataflowEndpoint} to generate the full URL to communicate with the Dataflow API.
-   */
-  @Description("The root URL for the Dataflow API. dataflowEndpoint can override this "
-      + "value if it contains an absolute URL, otherwise apiRootUrl will be combined with "
-      + "dataflowEndpoint to generate the full URL to communicate with the Dataflow API.")
-  @Default.String(Dataflow.DEFAULT_ROOT_URL)
-  String getApiRootUrl();
-  void setApiRootUrl(String value);
-
-  /**
-   * Dataflow endpoint to use.
-   *
-   * <p>Defaults to the current version of the Google Cloud Dataflow
-   * API, at the time the current SDK version was released.
-   *
-   * <p>If the string contains "://", then this is treated as a URL,
-   * otherwise {@link #getApiRootUrl()} is used as the root
-   * URL.
-   */
-  @Description("The URL for the Dataflow API. If the string contains \"://\", this"
-      + " will be treated as the entire URL, otherwise will be treated relative to apiRootUrl.")
-  @Default.String(Dataflow.DEFAULT_SERVICE_PATH)
-  String getDataflowEndpoint();
-  void setDataflowEndpoint(String value);
-
-  /**
-   * The path to write the translated Dataflow job specification out to
-   * at job submission time. The Dataflow job specification will be represented in JSON
-   * format.
-   */
-  @Description("The path to write the translated Dataflow job specification out to "
-      + "at job submission time. The Dataflow job specification will be represented in JSON "
-      + "format.")
-  String getDataflowJobFile();
-  void setDataflowJobFile(String value);
-
-  /**
-   * The class of the validator that should be created and used to validate paths.
-   * If pathValidator has not been set explicitly, an instance of this class will be
-   * constructed and used as the path validator.
-   */
-  @Description("The class of the validator that should be created and used to validate paths. "
-      + "If pathValidator has not been set explicitly, an instance of this class will be "
-      + "constructed and used as the path validator.")
-  @Default.Class(DataflowPathValidator.class)
-  Class<? extends PathValidator> getPathValidatorClass();
-  void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
-
-  /**
-   * The path validator instance that should be used to validate paths.
-   * If no path validator has been set explicitly, the default is to use the instance factory that
-   * constructs a path validator based upon the currently set pathValidatorClass.
-   */
-  @JsonIgnore
-  @Description("The path validator instance that should be used to validate paths. "
-      + "If no path validator has been set explicitly, the default is to use the instance factory "
-      + "that constructs a path validator based upon the currently set pathValidatorClass.")
-  @Default.InstanceFactory(PathValidatorFactory.class)
-  PathValidator getPathValidator();
-  void setPathValidator(PathValidator validator);
-
-  /**
-   * The class responsible for staging resources to be accessible by workers
-   * during job execution. If stager has not been set explicitly, an instance of this class
-   * will be created and used as the resource stager.
-   */
-  @Description("The class of the stager that should be created and used to stage resources. "
-      + "If stager has not been set explicitly, an instance of the this class will be created "
-      + "and used as the resource stager.")
-  @Default.Class(GcsStager.class)
-  Class<? extends Stager> getStagerClass();
-  void setStagerClass(Class<? extends Stager> stagerClass);
-
-  /**
-   * The resource stager instance that should be used to stage resources.
-   * If no stager has been set explicitly, the default is to use the instance factory
-   * that constructs a resource stager based upon the currently set stagerClass.
-   */
-  @JsonIgnore
-  @Description("The resource stager instance that should be used to stage resources. "
-      + "If no stager has been set explicitly, the default is to use the instance factory "
-      + "that constructs a resource stager based upon the currently set stagerClass.")
-  @Default.InstanceFactory(StagerFactory.class)
-  Stager getStager();
-  void setStager(Stager stager);
-
-  /**
-   * An instance of the Dataflow client. Defaults to creating a Dataflow client
-   * using the current set of options.
-   */
-  @JsonIgnore
-  @Description("An instance of the Dataflow client. Defaults to creating a Dataflow client "
-      + "using the current set of options.")
-  @Default.InstanceFactory(DataflowClientFactory.class)
-  Dataflow getDataflowClient();
-  void setDataflowClient(Dataflow value);
-
-  /** Returns the default Dataflow client built from the passed in PipelineOptions. */
-  public static class DataflowClientFactory implements DefaultValueFactory<Dataflow> {
-    @Override
-    public Dataflow create(PipelineOptions options) {
-        return DataflowTransport.newDataflowClient(
-            options.as(DataflowPipelineOptions.class)).build();
-    }
-  }
-
-  /**
-   * Whether to update the currently running pipeline with the same name as this one.
-   *
-   * @deprecated This property is replaced by {@link DataflowPipelineOptions#getUpdate()}
-   */
-  @Deprecated
-  @Description("If set, replace the existing pipeline with the name specified by --jobName with "
-      + "this pipeline, preserving state.")
-  boolean getUpdate();
-  @Deprecated
-  void setUpdate(boolean value);
-
-  /**
-   * Mapping of old PTranform names to new ones, specified as JSON
-   * <code>{"oldName":"newName",...}</code>. To mark a transform as deleted, make newName the
-   * empty string.
-   */
-  @JsonIgnore
-  @Description(
-      "Mapping of old PTranform names to new ones, specified as JSON "
-      + "{\"oldName\":\"newName\",...}. To mark a transform as deleted, make newName the empty "
-      + "string.")
-  Map<String, String> getTransformNameMapping();
-  void setTransformNameMapping(Map<String, String> value);
-
-  /**
-   * Custom windmill_main binary to use with the streaming runner.
-   */
-  @Description("Custom windmill_main binary to use with the streaming runner")
-  String getOverrideWindmillBinary();
-  void setOverrideWindmillBinary(String value);
-
-  /**
-   * Number of threads to use on the Dataflow worker harness. If left unspecified,
-   * the Dataflow service will compute an appropriate number of threads to use.
-   */
-  @Description("Number of threads to use on the Dataflow worker harness. If left unspecified, "
-      + "the Dataflow service will compute an appropriate number of threads to use.")
-  int getNumberOfWorkerHarnessThreads();
-  void setNumberOfWorkerHarnessThreads(int value);
-
-  /**
-   * If {@literal true}, save a heap dump before killing a thread or process which is GC
-   * thrashing or out of memory. The location of the heap file will either be echoed back
-   * to the user, or the user will be given the opportunity to download the heap file.
-   *
-   * <p>
-   * CAUTION: Heap dumps can of comparable size to the default boot disk. Consider increasing
-   * the boot disk size before setting this flag to true.
-   */
-  @Description("If {@literal true}, save a heap dump before killing a thread or process "
-      + "which is GC thrashing or out of memory.")
-  boolean getDumpHeapOnOOM();
-  void setDumpHeapOnOOM(boolean dumpHeapBeforeExit);
-
-  /**
-   * Creates a {@link PathValidator} object using the class specified in
-   * {@link #getPathValidatorClass()}.
-   */
-  public static class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
-      @Override
-      public PathValidator create(PipelineOptions options) {
-      DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
-      return InstanceBuilder.ofType(PathValidator.class)
-          .fromClass(debugOptions.getPathValidatorClass())
-          .fromFactoryMethod("fromOptions")
-          .withArg(PipelineOptions.class, options)
-          .build();
-    }
-  }
-
-  /**
-   * Creates a {@link Stager} object using the class specified in
-   * {@link #getStagerClass()}.
-   */
-  public static class StagerFactory implements DefaultValueFactory<Stager> {
-      @Override
-      public Stager create(PipelineOptions options) {
-      DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
-      return InstanceBuilder.ofType(Stager.class)
-          .fromClass(debugOptions.getStagerClass())
-          .fromFactoryMethod("fromOptions")
-          .withArg(PipelineOptions.class, options)
-          .build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java
deleted file mode 100644
index dbfafd1..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java
+++ /dev/null
@@ -1,115 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.runners.DataflowPipeline;
-import com.google.common.base.MoreObjects;
-
-import org.joda.time.DateTimeUtils;
-import org.joda.time.DateTimeZone;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-/**
- * Options that can be used to configure the {@link DataflowPipeline}.
- */
-@Description("Options that configure the Dataflow pipeline.")
-public interface DataflowPipelineOptions extends
-    PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions,
-    DataflowPipelineWorkerPoolOptions, BigQueryOptions,
-    GcsOptions, StreamingOptions, CloudDebuggerOptions, DataflowWorkerLoggingOptions,
-    DataflowProfilingOptions, PubsubOptions {
-
-  @Description("Project id. Required when running a Dataflow in the cloud. "
-      + "See https://cloud.google.com/storage/docs/projects for further details.")
-  @Override
-  @Validation.Required
-  @Default.InstanceFactory(DefaultProjectFactory.class)
-  String getProject();
-  @Override
-  void setProject(String value);
-
-  /**
-   * GCS path for staging local files, e.g. gs://bucket/object
-   *
-   * <p>Must be a valid Cloud Storage URL, beginning with the prefix "gs://"
-   *
-   * <p>At least one of {@link PipelineOptions#getTempLocation()} or {@link #getStagingLocation()}
-   * must be set. If {@link #getStagingLocation()} is not set, then the Dataflow
-   * pipeline defaults to using {@link PipelineOptions#getTempLocation()}.
-   */
-  @Description("GCS path for staging local files, e.g. \"gs://bucket/object\". "
-      + "Must be a valid Cloud Storage URL, beginning with the prefix \"gs://\". "
-      + "At least one of stagingLocation or tempLocation must be set. If stagingLocation is unset, "
-      + "defaults to using tempLocation.")
-  String getStagingLocation();
-  void setStagingLocation(String value);
-
-  /**
-   * The Dataflow job name is used as an idempotence key within the Dataflow service.
-   * If there is an existing job that is currently active, another active job with the same
-   * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.
-   */
-  @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. "
-      + "If there is an existing job that is currently active, another active job with the same "
-      + "name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.")
-  @Default.InstanceFactory(JobNameFactory.class)
-  String getJobName();
-  void setJobName(String value);
-
-  /**
-   * Whether to update the currently running pipeline with the same name as this one.
-   */
-  @Override
-  @SuppressWarnings("deprecation") // base class member deprecated in favor of this one.
-  @Description(
-      "If set, replace the existing pipeline with the name specified by --jobName with "
-          + "this pipeline, preserving state.")
-  boolean getUpdate();
-  @Override
-  @SuppressWarnings("deprecation") // base class member deprecated in favor of this one.
-  void setUpdate(boolean value);
-
-  /**
-   * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the
-   * local system user name (if available), and the current time. The normalization makes sure that
-   * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40
-   * characters.
-   *
-   * <p>This job name factory is only able to generate one unique name per second per application
-   * and user combination.
-   */
-  public static class JobNameFactory implements DefaultValueFactory<String> {
-    private static final DateTimeFormatter FORMATTER =
-        DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC);
-
-    @Override
-    public String create(PipelineOptions options) {
-      String appName = options.as(ApplicationNameOptions.class).getAppName();
-      String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow"
-          : appName.toLowerCase()
-                   .replaceAll("[^a-z0-9]", "0")
-                   .replaceAll("^[^a-z]", "a");
-      String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), "");
-      String normalizedUserName = userName.toLowerCase()
-                                          .replaceAll("[^a-z0-9]", "0");
-      String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis());
-      return normalizedAppName + "-" + normalizedUserName + "-" + datePart;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
deleted file mode 100644
index 44a9b00..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
+++ /dev/null
@@ -1,258 +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 com.google.cloud.dataflow.sdk.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.List;
-
-/**
- * Options that are used to configure the Dataflow pipeline worker pool.
- */
-@Description("Options that are used to configure the Dataflow pipeline worker pool.")
-public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions {
-  /**
-   * Number of workers to use when executing the Dataflow job. Note that selection of an autoscaling
-   * algorithm other then {@code NONE} will affect the size of the worker pool. If left unspecified,
-   * the Dataflow service will determine the number of workers.
-   */
-  @Description("Number of workers to use when executing the Dataflow job. Note that "
-      + "selection of an autoscaling algorithm other then \"NONE\" will affect the "
-      + "size of the worker pool. If left unspecified, the Dataflow service will "
-      + "determine the number of workers.")
-  int getNumWorkers();
-  void setNumWorkers(int value);
-
-  /**
-   * Type of autoscaling algorithm to use.
-   */
-  @Experimental(Experimental.Kind.AUTOSCALING)
-  public enum AutoscalingAlgorithmType {
-    /** Use numWorkers machines. Do not autoscale the worker pool. */
-    NONE("AUTOSCALING_ALGORITHM_NONE"),
-
-    @Deprecated
-    BASIC("AUTOSCALING_ALGORITHM_BASIC"),
-
-    /** Autoscale the workerpool based on throughput (up to maxNumWorkers). */
-    THROUGHPUT_BASED("AUTOSCALING_ALGORITHM_BASIC");
-
-    private final String algorithm;
-
-    private AutoscalingAlgorithmType(String algorithm) {
-      this.algorithm = algorithm;
-    }
-
-    /** Returns the string representation of this type. */
-    public String getAlgorithm() {
-      return this.algorithm;
-    }
-  }
-
-  /**
-   * [Experimental] The autoscaling algorithm to use for the workerpool.
-   *
-   * <ul>
-   *   <li>NONE: does not change the size of the worker pool.</li>
-   *   <li>BASIC: autoscale the worker pool size up to maxNumWorkers until the job completes.</li>
-   *   <li>THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).
-   *   </li>
-   * </ul>
-   */
-  @Description("[Experimental] The autoscaling algorithm to use for the workerpool. "
-      + "NONE: does not change the size of the worker pool. "
-      + "BASIC (deprecated): autoscale the worker pool size up to maxNumWorkers until the job "
-      + "completes. "
-      + "THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).")
-  @Experimental(Experimental.Kind.AUTOSCALING)
-  AutoscalingAlgorithmType getAutoscalingAlgorithm();
-  void setAutoscalingAlgorithm(AutoscalingAlgorithmType value);
-
-  /**
-   * The maximum number of workers to use for the workerpool. This options limits the size of the
-   * workerpool for the lifetime of the job, including
-   * <a href="https://cloud.google.com/dataflow/pipelines/updating-a-pipeline">pipeline updates</a>.
-   * If left unspecified, the Dataflow service will compute a ceiling.
-   */
-  @Description("The maximum number of workers to use for the workerpool. This options limits the "
-      + "size of the workerpool for the lifetime of the job, including pipeline updates. "
-      + "If left unspecified, the Dataflow service will compute a ceiling.")
-  int getMaxNumWorkers();
-  void setMaxNumWorkers(int value);
-
-  /**
-   * Remote worker disk size, in gigabytes, or 0 to use the default size.
-   */
-  @Description("Remote worker disk size, in gigabytes, or 0 to use the default size.")
-  int getDiskSizeGb();
-  void setDiskSizeGb(int value);
-
-  /**
-   * Docker container image that executes Dataflow worker harness, residing in Google Container
-   * Registry.
-   */
-  @Default.InstanceFactory(WorkerHarnessContainerImageFactory.class)
-  @Description("Docker container image that executes Dataflow worker harness, residing in Google "
-      + " Container Registry.")
-  @Hidden
-  String getWorkerHarnessContainerImage();
-  void setWorkerHarnessContainerImage(String value);
-
-  /**
-   * Returns the default Docker container image that executes Dataflow worker harness, residing in
-   * Google Container Registry.
-   */
-  public static class WorkerHarnessContainerImageFactory
-      implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-      if (dataflowOptions.isStreaming()) {
-        return DataflowPipelineRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
-      } else {
-        return DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
-      }
-    }
-  }
-
-  /**
-   * GCE <a href="https://cloud.google.com/compute/docs/networking">network</a> for launching
-   * workers.
-   *
-   * <p>Default is up to the Dataflow service.
-   */
-  @Description("GCE network for launching workers. For more information, see the reference "
-      + "documentation https://cloud.google.com/compute/docs/networking. "
-      + "Default is up to the Dataflow service.")
-  String getNetwork();
-  void setNetwork(String value);
-
-  /**
-   * GCE <a href="https://cloud.google.com/compute/docs/networking">subnetwork</a> for launching
-   * workers.
-   *
-   * <p>Default is up to the Dataflow service. Expected format is
-   * regions/REGION/subnetworks/SUBNETWORK.
-   *
-   * <p>You may also need to specify network option.
-   */
-  @Description("GCE subnetwork for launching workers. For more information, see the reference "
-      + "documentation https://cloud.google.com/compute/docs/networking. "
-      + "Default is up to the Dataflow service.")
-  String getSubnetwork();
-  void setSubnetwork(String value);
-
-  /**
-   * GCE <a href="https://developers.google.com/compute/docs/zones"
-   * >availability zone</a> for launching workers.
-   *
-   * <p>Default is up to the Dataflow service.
-   */
-  @Description("GCE availability zone for launching workers. See "
-      + "https://developers.google.com/compute/docs/zones for a list of valid options. "
-      + "Default is up to the Dataflow service.")
-  String getZone();
-  void setZone(String value);
-
-  /**
-   * Machine type to create Dataflow worker VMs as.
-   *
-   * <p>See <a href="https://cloud.google.com/compute/docs/machine-types">GCE machine types</a>
-   * for a list of valid options.
-   *
-   * <p>If unset, the Dataflow service will choose a reasonable default.
-   */
-  @Description("Machine type to create Dataflow worker VMs as. See "
-      + "https://cloud.google.com/compute/docs/machine-types for a list of valid options. "
-      + "If unset, the Dataflow service will choose a reasonable default.")
-  String getWorkerMachineType();
-  void setWorkerMachineType(String value);
-
-  /**
-   * The policy for tearing down the workers spun up by the service.
-   */
-  public enum TeardownPolicy {
-    /**
-     * All VMs created for a Dataflow job are deleted when the job finishes, regardless of whether
-     * it fails or succeeds.
-     */
-    TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"),
-    /**
-     * All VMs created for a Dataflow job are left running when the job finishes, regardless of
-     * whether it fails or succeeds.
-     */
-    TEARDOWN_NEVER("TEARDOWN_NEVER"),
-    /**
-     * All VMs created for a Dataflow job are deleted when the job succeeds, but are left running
-     * when it fails. (This is typically used for debugging failing jobs by SSHing into the
-     * workers.)
-     */
-    TEARDOWN_ON_SUCCESS("TEARDOWN_ON_SUCCESS");
-
-    private final String teardownPolicy;
-
-    private TeardownPolicy(String teardownPolicy) {
-      this.teardownPolicy = teardownPolicy;
-    }
-
-    public String getTeardownPolicyName() {
-      return this.teardownPolicy;
-    }
-  }
-
-  /**
-   * The teardown policy for the VMs.
-   *
-   * <p>If unset, the Dataflow service will choose a reasonable default.
-   */
-  @Description("The teardown policy for the VMs. If unset, the Dataflow service will "
-      + "choose a reasonable default.")
-  TeardownPolicy getTeardownPolicy();
-  void setTeardownPolicy(TeardownPolicy value);
-
-  /**
-   * List of local files to make available to workers.
-   *
-   * <p>Files are placed on the worker's classpath.
-   *
-   * <p>The default value is the list of jars from the main program's classpath.
-   */
-  @Description("Files to stage on GCS and make available to workers. "
-      + "Files are placed on the worker's classpath. "
-      + "The default value is all files from the classpath.")
-  @JsonIgnore
-  List<String> getFilesToStage();
-  void setFilesToStage(List<String> value);
-
-  /**
-   * Specifies what type of persistent disk should be used. The value should be a full or partial
-   * URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For
-   * more information, see the
-   * <a href="https://cloud.google.com/compute/docs/reference/latest/diskTypes">API reference
-   * documentation for DiskTypes</a>.
-   */
-  @Description("Specifies what type of persistent disk should be used. The value should be a full "
-      + "or partial URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For "
-      + "more information, see the API reference documentation for DiskTypes: "
-      + "https://cloud.google.com/compute/docs/reference/latest/diskTypes")
-  String getWorkerDiskType();
-  void setWorkerDiskType(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptions.java
deleted file mode 100644
index 3cd7b03..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowProfilingOptions.java
+++ /dev/null
@@ -1,48 +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 com.google.cloud.dataflow.sdk.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import java.util.HashMap;
-
-/**
- * Options for controlling profiling of pipeline execution.
- */
-@Description("[Experimental] Used to configure profiling of the Dataflow pipeline")
-@Experimental
-@Hidden
-public interface DataflowProfilingOptions {
-
-  @Description("Whether to periodically dump profiling information to local disk.\n"
-      + "WARNING: Enabling this option may fill local disk with profiling information.")
-  boolean getEnableProfilingAgent();
-  void setEnableProfilingAgent(boolean enabled);
-
-  @Description(
-      "[INTERNAL] Additional configuration for the profiling agent. Not typically necessary.")
-  @Hidden
-  DataflowProfilingAgentConfiguration getProfilingAgentConfiguration();
-  void setProfilingAgentConfiguration(DataflowProfilingAgentConfiguration configuration);
-
-  /**
-   * Configuration the for profiling agent.
-   */
-  public static class DataflowProfilingAgentConfiguration extends HashMap<String, Object> {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java
deleted file mode 100644
index 7705b66..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java
+++ /dev/null
@@ -1,51 +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 com.google.cloud.dataflow.sdk.options;
-
-/**
- * Options that are used exclusively within the Dataflow worker harness.
- * These options have no effect at pipeline creation time.
- */
-@Description("[Internal] Options that are used exclusively within the Dataflow worker harness. "
-    + "These options have no effect at pipeline creation time.")
-@Hidden
-public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions {
-  /**
-   * The identity of the worker running this pipeline.
-   */
-  @Description("The identity of the worker running this pipeline.")
-  String getWorkerId();
-  void setWorkerId(String value);
-
-  /**
-   * The identity of the Dataflow job.
-   */
-  @Description("The identity of the Dataflow job.")
-  String getJobId();
-  void setJobId(String value);
-
-  /**
-   * The size of the worker's in-memory cache, in megabytes.
-   *
-   * <p>Currently, this cache is used for storing read values of side inputs.
-   */
-  @Description("The size of the worker's in-memory cache, in megabytes.")
-  @Default.Integer(100)
-  Integer getWorkerCacheMb();
-  void setWorkerCacheMb(Integer value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java
deleted file mode 100644
index ebd42d9..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java
+++ /dev/null
@@ -1,155 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Options that are used to control logging configuration on the Dataflow worker.
- */
-@Description("Options that are used to control logging configuration on the Dataflow worker.")
-public interface DataflowWorkerLoggingOptions extends PipelineOptions {
-  /**
-   * The set of log levels that can be used on the Dataflow worker.
-   */
-  public enum Level {
-    DEBUG, ERROR, INFO, TRACE, WARN
-  }
-
-  /**
-   * This option controls the default log level of all loggers without a log level override.
-   */
-  @Description("Controls the default log level of all loggers without a log level override.")
-  @Default.Enum("INFO")
-  Level getDefaultWorkerLogLevel();
-  void setDefaultWorkerLogLevel(Level level);
-
-  /**
-   * This option controls the log levels for specifically named loggers.
-   *
-   * <p>Later options with equivalent names override earlier options.
-   *
-   * <p>See {@link WorkerLogLevelOverrides} for more information on how to configure logging
-   * on a per {@link Class}, {@link Package}, or name basis. If used from the command line,
-   * the expected format is {"Name":"Level",...}, further details on
-   * {@link WorkerLogLevelOverrides#from}.
-   */
-  @Description("This option controls the log levels for specifically named loggers. "
-      + "The expected format is {\"Name\":\"Level\",...}. The Dataflow worker uses "
-      + "java.util.logging, which supports a logging hierarchy based off of names that are '.' "
-      + "separated. For example, by specifying the value {\"a.b.c.Foo\":\"DEBUG\"}, the logger "
-      + "for the class 'a.b.c.Foo' will be configured to output logs at the DEBUG level. "
-      + "Similarly, by specifying the value {\"a.b.c\":\"WARN\"}, all loggers underneath the "
-      + "'a.b.c' package will be configured to output logs at the WARN level. Also, note that "
-      + "when multiple overrides are specified, the exact name followed by the closest parent "
-      + "takes precedence.")
-  WorkerLogLevelOverrides getWorkerLogLevelOverrides();
-  void setWorkerLogLevelOverrides(WorkerLogLevelOverrides value);
-
-  /**
-   * Defines a log level override for a specific class, package, or name.
-   *
-   * <p>{@code java.util.logging} is used on the Dataflow worker harness and supports
-   * a logging hierarchy based off of names that are "." separated. It is a common
-   * pattern to have the logger for a given class share the same name as the class itself.
-   * Given the classes {@code a.b.c.Foo}, {@code a.b.c.Xyz}, and {@code a.b.Bar}, with
-   * loggers named {@code "a.b.c.Foo"}, {@code "a.b.c.Xyz"}, and {@code "a.b.Bar"} respectively,
-   * we can override the log levels:
-   * <ul>
-   *    <li>for {@code Foo} by specifying the name {@code "a.b.c.Foo"} or the {@link Class}
-   *    representing {@code a.b.c.Foo}.
-   *    <li>for {@code Foo}, {@code Xyz}, and {@code Bar} by specifying the name {@code "a.b"} or
-   *    the {@link Package} representing {@code a.b}.
-   *    <li>for {@code Foo} and {@code Bar} by specifying both of their names or classes.
-   * </ul>
-   * Note that by specifying multiple overrides, the exact name followed by the closest parent
-   * takes precedence.
-   */
-  public static class WorkerLogLevelOverrides extends HashMap<String, Level> {
-    /**
-     * Overrides the default log level for the passed in class.
-     *
-     * <p>This is equivalent to calling
-     * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
-     * 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.");
-      addOverrideForName(klass.getName(), level);
-      return this;
-    }
-
-    /**
-     * Overrides the default log level for the passed in package.
-     *
-     * <p>This is equivalent to calling
-     * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
-     * 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.");
-      addOverrideForName(pkg.getName(), level);
-      return this;
-    }
-
-    /**
-     * Overrides the default log level for the passed in name.
-     *
-     * <p>Note that because of the hierarchical nature of logger names, this will
-     * override the log level of all loggers that have the passed in name or
-     * 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,
-          "Expected level to be one of %s.", Arrays.toString(Level.values()));
-      put(name, level);
-      return this;
-    }
-
-    /**
-     * Expects a map keyed by logger {@code Name}s with values representing {@code Level}s.
-     * The {@code Name} generally represents the fully qualified Java
-     * {@link Class#getName() class name}, or fully qualified Java
-     * {@link Package#getName() package name}, or custom logger name. The {@code Level}
-     * represents the log level and must be one of {@link Level}.
-     */
-    @JsonCreator
-    public static WorkerLogLevelOverrides from(Map<String, String> values) {
-      Preconditions.checkNotNull(values, "Expected values to be not null.");
-      WorkerLogLevelOverrides overrides = new WorkerLogLevelOverrides();
-      for (Map.Entry<String, String> entry : values.entrySet()) {
-        try {
-          overrides.addOverrideForName(entry.getKey(), Level.valueOf(entry.getValue()));
-        } catch (IllegalArgumentException e) {
-          throw new IllegalArgumentException(String.format(
-              "Unsupported log level '%s' requested for %s. Must be one of %s.",
-              entry.getValue(), entry.getKey(), Arrays.toString(Level.values())));
-        }
-
-      }
-      return overrides;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java
deleted file mode 100644
index 9096c2b..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java
+++ /dev/null
@@ -1,186 +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 com.google.cloud.dataflow.sdk.runners;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link PipelineRunner} that's like {@link DataflowPipelineRunner}
- * but that waits for the launched job to finish.
- *
- * <p>Prints out job status updates and console messages while it waits.
- *
- * <p>Returns the final job state, or throws an exception if the job
- * fails or cannot be monitored.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code BlockingDataflowPipelineRunner}, the Google cloud services account and the Google compute
- * engine service account of the GCP project running the Dataflow Job will need access to the
- * corresponding source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class BlockingDataflowPipelineRunner extends
-    PipelineRunner<DataflowPipelineJob> {
-  private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class);
-
-  // Defaults to an infinite wait period.
-  // TODO: make this configurable after removal of option map.
-  private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
-
-  private final DataflowPipelineRunner dataflowPipelineRunner;
-  private final BlockingDataflowPipelineOptions options;
-
-  protected BlockingDataflowPipelineRunner(
-      DataflowPipelineRunner internalRunner,
-      BlockingDataflowPipelineOptions options) {
-    this.dataflowPipelineRunner = internalRunner;
-    this.options = options;
-  }
-
-  /**
-   * Constructs a runner from the provided options.
-   */
-  public static BlockingDataflowPipelineRunner fromOptions(
-      PipelineOptions options) {
-    BlockingDataflowPipelineOptions dataflowOptions =
-        PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
-    DataflowPipelineRunner dataflowPipelineRunner =
-        DataflowPipelineRunner.fromOptions(dataflowOptions);
-
-    return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, dataflowOptions);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws DataflowJobExecutionException if there is an exception during job execution.
-   * @throws DataflowServiceException if there is an exception retrieving information about the job.
-   */
-  @Override
-  public DataflowPipelineJob run(Pipeline p) {
-    final DataflowPipelineJob job = dataflowPipelineRunner.run(p);
-
-    // We ignore the potential race condition here (Ctrl-C after job submission but before the
-    // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
-    // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
-    // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
-    // etc. If the user wants to verify the job was cancelled they should look at the job status.
-    Thread shutdownHook = new Thread() {
-      @Override
-      public void run() {
-        LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
-            + "To cancel the job in the cloud, run:\n> {}",
-            MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
-      }
-    };
-
-    try {
-      Runtime.getRuntime().addShutdownHook(shutdownHook);
-
-      @Nullable
-      State result;
-      try {
-        result = job.waitToFinish(
-            BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
-            new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
-      } catch (IOException | InterruptedException ex) {
-        if (ex instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
-        throw new DataflowServiceException(
-            job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
-      }
-
-      if (result == null) {
-        throw new DataflowServiceException(
-            job, "Timed out while retrieving status for job " + job.getJobId());
-      }
-
-      LOG.info("Job finished with status {}", result);
-      if (!result.isTerminal()) {
-        throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
-            + ", got " + result);
-      }
-
-      if (result == State.DONE) {
-        return job;
-      } else if (result == State.UPDATED) {
-        DataflowPipelineJob newJob = job.getReplacedByJob();
-        LOG.info("Job {} has been updated and is running as the new job with id {}."
-            + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
-            job.getJobId(),
-            newJob.getJobId(),
-            MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
-        throw new DataflowJobUpdatedException(
-            job,
-            String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
-            newJob);
-      } else if (result == State.CANCELLED) {
-        String message = String.format("Job %s cancelled by user", job.getJobId());
-        LOG.info(message);
-        throw new DataflowJobCancelledException(job, message);
-      } else {
-        throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
-            + " failed with status " + result);
-      }
-    } finally {
-      Runtime.getRuntime().removeShutdownHook(shutdownHook);
-    }
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    return dataflowPipelineRunner.apply(transform, input);
-  }
-
-  /**
-   * Sets callbacks to invoke during execution. See {@link DataflowPipelineRunnerHooks}.
-   */
-  @Experimental
-  public void setHooks(DataflowPipelineRunnerHooks hooks) {
-    this.dataflowPipelineRunner.setHooks(hooks);
-  }
-
-  @Override
-  public String toString() {
-    return "BlockingDataflowPipelineRunner#" + options.getJobName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyExistsException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyExistsException.java
deleted file mode 100644
index 4a4f100..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyExistsException.java
+++ /dev/null
@@ -1,35 +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 com.google.cloud.dataflow.sdk.runners;
-
-/**
- * An exception that is thrown if the unique job name constraint of the Dataflow
- * service is broken because an existing job with the same job name is currently active.
- * The {@link DataflowPipelineJob} contained within this exception contains information
- * about the pre-existing job.
- */
-public class DataflowJobAlreadyExistsException extends DataflowJobException {
-  /**
-   * Create a new {@code DataflowJobAlreadyExistsException} with the specified {@link
-   * DataflowPipelineJob} and message.
-   */
-  public DataflowJobAlreadyExistsException(
-      DataflowPipelineJob job, String message) {
-    super(job, message, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyUpdatedException.java
deleted file mode 100644
index 1f52c6a..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobAlreadyUpdatedException.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.runners;
-
-/**
- * An exception that is thrown if the existing job has already been updated within the Dataflow
- * service and is no longer able to be updated. The {@link DataflowPipelineJob} contained within
- * this exception contains information about the pre-existing updated job.
- */
-public class DataflowJobAlreadyUpdatedException extends DataflowJobException {
-  /**
-   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
-   * DataflowPipelineJob} and message.
-   */
-  public DataflowJobAlreadyUpdatedException(
-      DataflowPipelineJob job, String message) {
-    super(job, message, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobCancelledException.java
deleted file mode 100644
index 495ca5a..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobCancelledException.java
+++ /dev/null
@@ -1,39 +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 com.google.cloud.dataflow.sdk.runners;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
- */
-public class DataflowJobCancelledException extends DataflowJobException {
-  /**
-   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
-   * DataflowPipelineJob} and message.
-   */
-  public DataflowJobCancelledException(DataflowPipelineJob job, String message) {
-    super(job, message, null);
-  }
-
-  /**
-   * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
-   * DataflowPipelineJob}, message, and cause.
-   */
-  public DataflowJobCancelledException(DataflowPipelineJob job, String message, Throwable cause) {
-    super(job, message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobException.java
deleted file mode 100644
index a22d13c..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobException.java
+++ /dev/null
@@ -1,41 +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 com.google.cloud.dataflow.sdk.runners;
-
-import java.util.Objects;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link RuntimeException} that contains information about a {@link DataflowPipelineJob}.
- */
-public abstract class DataflowJobException extends RuntimeException {
-  private final DataflowPipelineJob job;
-
-  DataflowJobException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
-    super(message, cause);
-    this.job = Objects.requireNonNull(job);
-  }
-
-  /**
-   * Returns the failed job.
-   */
-  public DataflowPipelineJob getJob() {
-    return job;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobExecutionException.java
deleted file mode 100644
index 3dbb007..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobExecutionException.java
+++ /dev/null
@@ -1,35 +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 com.google.cloud.dataflow.sdk.runners;
-
-import javax.annotation.Nullable;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} fails during execution, and
- * provides access to the failed job.
- */
-public class DataflowJobExecutionException extends DataflowJobException {
-  DataflowJobExecutionException(DataflowPipelineJob job, String message) {
-    this(job, message, null);
-  }
-
-  DataflowJobExecutionException(
-      DataflowPipelineJob job, String message, @Nullable Throwable cause) {
-    super(job, message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobUpdatedException.java
deleted file mode 100644
index 72deb45..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowJobUpdatedException.java
+++ /dev/null
@@ -1,52 +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 com.google.cloud.dataflow.sdk.runners;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
- */
-public class DataflowJobUpdatedException extends DataflowJobException {
-  private DataflowPipelineJob replacedByJob;
-
-  /**
-   * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
-   * DataflowPipelineJob}, message, and replacement {@link DataflowPipelineJob}.
-   */
-  public DataflowJobUpdatedException(
-      DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob) {
-    this(job, message, replacedByJob, null);
-  }
-
-  /**
-   * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
-   * DataflowPipelineJob}, message, replacement {@link DataflowPipelineJob}, and cause.
-   */
-  public DataflowJobUpdatedException(
-      DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob, Throwable cause) {
-    super(job, message, cause);
-    this.replacedByJob = replacedByJob;
-  }
-
-  /**
-   * The new job that replaces the job terminated with this exception.
-   */
-  public DataflowPipelineJob getReplacedByJob() {
-    return replacedByJob;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java
deleted file mode 100644
index 7b97c7d..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java
+++ /dev/null
@@ -1,60 +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 com.google.cloud.dataflow.sdk.runners;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-
-/**
- * A {@link DataflowPipeline} is a {@link Pipeline} that returns a
- * {@link DataflowPipelineJob} when it is
- * {@link com.google.cloud.dataflow.sdk.Pipeline#run()}.
- *
- * <p>This is not intended for use by users of Cloud Dataflow.
- * Instead, use {@link Pipeline#create(PipelineOptions)} to initialize a
- * {@link Pipeline}.
- */
-public class DataflowPipeline extends Pipeline {
-
-  /**
-   * Creates and returns a new {@link DataflowPipeline} instance for tests.
-   */
-  public static DataflowPipeline create(DataflowPipelineOptions options) {
-    return new DataflowPipeline(options);
-  }
-
-  private DataflowPipeline(DataflowPipelineOptions options) {
-    super(DataflowPipelineRunner.fromOptions(options), options);
-  }
-
-  @Override
-  public DataflowPipelineJob run() {
-    return (DataflowPipelineJob) super.run();
-  }
-
-  @Override
-  public DataflowPipelineRunner getRunner() {
-    return (DataflowPipelineRunner) super.getRunner();
-  }
-
-  @Override
-  public String toString() {
-    return "DataflowPipeline#" + getOptions().as(DataflowPipelineOptions.class).getJobName();
-  }
-}


[24/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
new file mode 100644
index 0000000..dd1b3c8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
@@ -0,0 +1,890 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
+import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary;
+import static com.google.cloud.dataflow.sdk.util.Structs.getString;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+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.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.OutputReference;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.Structs;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentMatcher;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for DataflowPipelineTranslator.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineTranslatorTest implements Serializable {
+
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+  // A Custom Mockito matcher for an initial Job that checks that all
+  // expected fields are set.
+  private static class IsValidCreateRequest extends ArgumentMatcher<Job> {
+    @Override
+    public boolean matches(Object o) {
+      Job job = (Job) o;
+      return job.getId() == null
+          && job.getProjectId() == null
+          && job.getName() != null
+          && job.getType() != null
+          && job.getEnvironment() != null
+          && job.getSteps() != null
+          && job.getCurrentState() == null
+          && job.getCurrentStateTime() == null
+          && job.getExecutionInfo() == null
+          && job.getCreateTime() == null;
+    }
+  }
+
+  private DataflowPipeline buildPipeline(DataflowPipelineOptions options) {
+    DataflowPipeline p = DataflowPipeline.create(options);
+
+    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
+        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+
+    return p;
+  }
+
+  private static Dataflow buildMockDataflow(
+      ArgumentMatcher<Job> jobMatcher) throws IOException {
+    Dataflow mockDataflowClient = mock(Dataflow.class);
+    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
+    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
+    Dataflow.Projects.Jobs.Create mockRequest = mock(
+        Dataflow.Projects.Jobs.Create.class);
+
+    when(mockDataflowClient.projects()).thenReturn(mockProjects);
+    when(mockProjects.jobs()).thenReturn(mockJobs);
+    when(mockJobs.create(eq("someProject"), argThat(jobMatcher)))
+        .thenReturn(mockRequest);
+
+    Job resultJob = new Job();
+    resultJob.setId("newid");
+    when(mockRequest.execute()).thenReturn(resultJob);
+    return mockDataflowClient;
+  }
+
+  private static DataflowPipelineOptions buildPipelineOptions() throws IOException {
+    GcsUtil mockGcsUtil = mock(GcsUtil.class);
+    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
+      @Override
+      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+      }
+    });
+    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
+    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
+
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setGcpCredential(new TestCredential());
+    options.setJobName("some-job-name");
+    options.setProject("some-project");
+    options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString());
+    options.setFilesToStage(new LinkedList<String>());
+    options.setDataflowClient(buildMockDataflow(new IsValidCreateRequest()));
+    options.setGcsUtil(mockGcsUtil);
+    return options;
+  }
+
+  @Test
+  public void testSettingOfSdkPipelineOptions() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setRunner(DataflowPipelineRunner.class);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    // Note that the contents of this materialized map may be changed by the act of reading an
+    // option, which will cause the default to get materialized whereas it would otherwise be
+    // left absent. It is permissible to simply alter this test to reflect current behavior.
+    Map<String, Object> settings = new HashMap<>();
+    settings.put("appName", "DataflowPipelineTranslatorTest");
+    settings.put("project", "some-project");
+    settings.put("pathValidatorClass", "com.google.cloud.dataflow.sdk.util.DataflowPathValidator");
+    settings.put("runner", "com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner");
+    settings.put("jobName", "some-job-name");
+    settings.put("tempLocation", "gs://somebucket/some/path");
+    settings.put("stagingLocation", "gs://somebucket/some/path/staging");
+    settings.put("stableUniqueNames", "WARNING");
+    settings.put("streaming", false);
+    settings.put("numberOfWorkerHarnessThreads", 0);
+    settings.put("experiments", null);
+
+    assertEquals(ImmutableMap.of("options", settings),
+        job.getEnvironment().getSdkPipelineOptions());
+  }
+
+  @Test
+  public void testNetworkConfig() throws IOException {
+    final String testNetwork = "test-network";
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setNetwork(testNetwork);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertEquals(testNetwork,
+        job.getEnvironment().getWorkerPools().get(0).getNetwork());
+  }
+
+  @Test
+  public void testNetworkConfigMissing() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertNull(job.getEnvironment().getWorkerPools().get(0).getNetwork());
+  }
+
+  @Test
+  public void testSubnetworkConfig() throws IOException {
+    final String testSubnetwork = "regions/REGION/subnetworks/SUBNETWORK";
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setSubnetwork(testSubnetwork);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertEquals(testSubnetwork,
+        job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
+  }
+
+  @Test
+  public void testSubnetworkConfigMissing() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertNull(job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
+  }
+
+  @Test
+  public void testScalingAlgorithmMissing() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    // Autoscaling settings are always set.
+    assertNull(
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getAlgorithm());
+    assertEquals(
+        0,
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getMaxNumWorkers()
+            .intValue());
+  }
+
+  @Test
+  public void testScalingAlgorithmNone() throws IOException {
+    final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling =
+        DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE;
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setAutoscalingAlgorithm(noScaling);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertEquals(
+        "AUTOSCALING_ALGORITHM_NONE",
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getAlgorithm());
+    assertEquals(
+        0,
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getMaxNumWorkers()
+            .intValue());
+  }
+
+  @Test
+  public void testMaxNumWorkersIsPassedWhenNoAlgorithmIsSet() throws IOException {
+    final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling = null;
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setMaxNumWorkers(42);
+    options.setAutoscalingAlgorithm(noScaling);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertNull(
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getAlgorithm());
+    assertEquals(
+        42,
+        job
+            .getEnvironment()
+            .getWorkerPools()
+            .get(0)
+            .getAutoscalingSettings()
+            .getMaxNumWorkers()
+            .intValue());
+  }
+
+  @Test
+  public void testZoneConfig() throws IOException {
+    final String testZone = "test-zone-1";
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setZone(testZone);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertEquals(testZone,
+        job.getEnvironment().getWorkerPools().get(0).getZone());
+  }
+
+  @Test
+  public void testWorkerMachineTypeConfig() throws IOException {
+    final String testMachineType = "test-machine-type";
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setWorkerMachineType(testMachineType);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+
+    WorkerPool workerPool = job.getEnvironment().getWorkerPools().get(0);
+    assertEquals(testMachineType, workerPool.getMachineType());
+  }
+
+  @Test
+  public void testDiskSizeGbConfig() throws IOException {
+    final Integer diskSizeGb = 1234;
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setDiskSizeGb(diskSizeGb);
+
+    DataflowPipeline p = buildPipeline(options);
+    p.traverseTopologically(new RecordingPipelineVisitor());
+    Job job =
+        DataflowPipelineTranslator.fromOptions(options)
+            .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList())
+            .getJob();
+
+    assertEquals(1, job.getEnvironment().getWorkerPools().size());
+    assertEquals(diskSizeGb,
+        job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb());
+  }
+
+  @Test
+  public void testPredefinedAddStep() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+    DataflowPipelineTranslator.registerTransformTranslator(
+        EmbeddedTransform.class, new EmbeddedTranslator());
+
+    // Create a predefined step using another pipeline
+    Step predefinedStep = createPredefinedStep();
+
+    // Create a pipeline that the predefined step will be embedded into
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
+        .apply(ParDo.of(new NoOpFn()))
+        .apply(new EmbeddedTransform(predefinedStep.clone()))
+        .apply(ParDo.of(new NoOpFn()));
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(4, steps.size());
+
+    // The input to the embedded step should match the output of the step before
+    Map<String, Object> step1Out = getOutputPortReference(steps.get(1));
+    Map<String, Object> step2In = getDictionary(
+        steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT);
+    assertEquals(step1Out, step2In);
+
+    // The output from the embedded step should match the input of the step after
+    Map<String, Object> step2Out = getOutputPortReference(steps.get(2));
+    Map<String, Object> step3In = getDictionary(
+        steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT);
+    assertEquals(step2Out, step3In);
+
+    // The step should not have been modified other than remapping the input
+    Step predefinedStepClone = predefinedStep.clone();
+    Step embeddedStepClone = steps.get(2).clone();
+    predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
+    embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
+    assertEquals(predefinedStepClone, embeddedStepClone);
+  }
+
+  /**
+   * Construct a OutputReference for the output of the step.
+   */
+  private static OutputReference getOutputPortReference(Step step) throws Exception {
+    // TODO: This should be done via a Structs accessor.
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> output =
+        (List<Map<String, Object>>) step.getProperties().get(PropertyNames.OUTPUT_INFO);
+    String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME);
+    return new OutputReference(step.getName(), outputTagId);
+  }
+
+  /**
+   * Returns a Step for a DoFn by creating and translating a pipeline.
+   */
+  private static Step createPredefinedStep() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    String stepName = "DoFn1";
+    pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
+        .apply(ParDo.of(new NoOpFn()).named(stepName))
+        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out"));
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    assertEquals(13, job.getSteps().size());
+    Step step = job.getSteps().get(1);
+    assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME));
+    return step;
+  }
+
+  private static class NoOpFn extends DoFn<String, String> {
+    @Override public void processElement(ProcessContext c) throws Exception {
+      c.output(c.element());
+    }
+  }
+
+  /**
+   * A placeholder transform that will be used to substitute a predefined Step.
+   */
+  private static class EmbeddedTransform
+      extends PTransform<PCollection<String>, PCollection<String>> {
+    private final Step step;
+
+    public EmbeddedTransform(Step step) {
+      this.step = step;
+    }
+
+    @Override
+    public PCollection<String> apply(PCollection<String> input) {
+      return PCollection.createPrimitiveOutputInternal(
+          input.getPipeline(),
+          WindowingStrategy.globalDefault(),
+          input.isBounded());
+    }
+
+    @Override
+    protected Coder<?> getDefaultOutputCoder() {
+      return StringUtf8Coder.of();
+    }
+  }
+
+  /**
+   * A TransformTranslator that adds the predefined Step using
+   * {@link TranslationContext#addStep} and remaps the input port reference.
+   */
+  private static class EmbeddedTranslator
+      implements DataflowPipelineTranslator.TransformTranslator<EmbeddedTransform> {
+    @Override public void translate(EmbeddedTransform transform, TranslationContext context) {
+      addObject(transform.step.getProperties(), PropertyNames.PARALLEL_INPUT,
+          context.asOutputReference(context.getInput(transform)));
+      context.addStep(transform, transform.step);
+    }
+  }
+
+  /**
+   * A composite transform that returns an output that is unrelated to
+   * the input.
+   */
+  private static class UnrelatedOutputCreator
+      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
+
+    @Override
+    public PCollection<Integer> apply(PCollection<Integer> input) {
+      // Apply an operation so that this is a composite transform.
+      input.apply(Count.<Integer>perElement());
+
+      // Return a value unrelated to the input.
+      return input.getPipeline().apply(Create.of(1, 2, 3, 4));
+    }
+
+    @Override
+    protected Coder<?> getDefaultOutputCoder() {
+      return VarIntCoder.of();
+    }
+  }
+
+  /**
+   * A composite transform that returns an output that is unbound.
+   */
+  private static class UnboundOutputCreator
+      extends PTransform<PCollection<Integer>, PDone> {
+
+    @Override
+    public PDone apply(PCollection<Integer> input) {
+      // Apply an operation so that this is a composite transform.
+      input.apply(Count.<Integer>perElement());
+
+      return PDone.in(input.getPipeline());
+    }
+
+    @Override
+    protected Coder<?> getDefaultOutputCoder() {
+      return VoidCoder.of();
+    }
+  }
+
+  /**
+   * A composite transform that returns a partially bound output.
+   *
+   * <p>This is not allowed and will result in a failure.
+   */
+  private static class PartiallyBoundOutputCreator
+      extends PTransform<PCollection<Integer>, PCollectionTuple> {
+
+    public final TupleTag<Integer> sumTag = new TupleTag<>("sum");
+    public final TupleTag<Void> doneTag = new TupleTag<>("done");
+
+    @Override
+    public PCollectionTuple apply(PCollection<Integer> input) {
+      PCollection<Integer> sum = input.apply(Sum.integersGlobally());
+
+      // Fails here when attempting to construct a tuple with an unbound object.
+      return PCollectionTuple.of(sumTag, sum)
+          .and(doneTag, PCollection.<Void>createPrimitiveOutputInternal(
+              input.getPipeline(),
+              WindowingStrategy.globalDefault(),
+              input.isBounded()));
+    }
+  }
+
+  @Test
+  public void testMultiGraphPipelineSerialization() throws IOException {
+    DataflowPipeline p = DataflowPipeline.create(buildPipelineOptions());
+
+    PCollection<Integer> input = p.begin()
+        .apply(Create.of(1, 2, 3));
+
+    input.apply(new UnrelatedOutputCreator());
+    input.apply(new UnboundOutputCreator());
+
+    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(
+        PipelineOptionsFactory.as(DataflowPipelineOptions.class));
+
+    // Check that translation doesn't fail.
+    t.translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList());
+  }
+
+  @Test
+  public void testPartiallyBoundFailure() throws IOException {
+    Pipeline p = DataflowPipeline.create(buildPipelineOptions());
+
+    PCollection<Integer> input = p.begin()
+        .apply(Create.of(1, 2, 3));
+
+    thrown.expect(IllegalStateException.class);
+    input.apply(new PartiallyBoundOutputCreator());
+
+    Assert.fail("Failure expected from use of partially bound output");
+  }
+
+  /**
+   * This tests a few corner cases that should not crash.
+   */
+  @Test
+  public void testGoodWildcards() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
+
+    applyRead(pipeline, "gs://bucket/foo");
+    applyRead(pipeline, "gs://bucket/foo/");
+    applyRead(pipeline, "gs://bucket/foo/*");
+    applyRead(pipeline, "gs://bucket/foo/?");
+    applyRead(pipeline, "gs://bucket/foo/[0-9]");
+    applyRead(pipeline, "gs://bucket/foo/*baz*");
+    applyRead(pipeline, "gs://bucket/foo/*baz?");
+    applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
+    applyRead(pipeline, "gs://bucket/foo/baz/*");
+    applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
+    applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
+    applyRead(pipeline, "gs://bucket/foo*/baz");
+    applyRead(pipeline, "gs://bucket/foo?/baz");
+    applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
+
+    // Check that translation doesn't fail.
+    t.translate(pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList());
+  }
+
+  private void applyRead(Pipeline pipeline, String path) {
+    pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
+  }
+
+  /**
+   * Recursive wildcards are not supported.
+   * This tests "**".
+   */
+  @Test
+  public void testBadWildcardRecursive() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
+
+    pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
+
+    // Check that translation does fail.
+    thrown.expectCause(Matchers.allOf(
+        instanceOf(IllegalArgumentException.class),
+        ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage"))));
+    t.translate(pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList());
+  }
+
+  @Test
+  public void testToSingletonTranslation() throws Exception {
+    // A "change detector" test that makes sure the translation
+    // of getting a PCollectionView<T> does not change
+    // in bad ways during refactor
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setExperiments(ImmutableList.of("disable_ism_side_input"));
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    pipeline.apply(Create.of(1))
+        .apply(View.<Integer>asSingleton());
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(2, steps.size());
+
+    Step createStep = steps.get(0);
+    assertEquals("CreateCollection", createStep.getKind());
+
+    Step collectionToSingletonStep = steps.get(1);
+    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+
+  }
+
+  @Test
+  public void testToIterableTranslation() throws Exception {
+    // A "change detector" test that makes sure the translation
+    // of getting a PCollectionView<Iterable<T>> does not change
+    // in bad ways during refactor
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setExperiments(ImmutableList.of("disable_ism_side_input"));
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    pipeline.apply(Create.of(1, 2, 3))
+        .apply(View.<Integer>asIterable());
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(2, steps.size());
+
+    Step createStep = steps.get(0);
+    assertEquals("CreateCollection", createStep.getKind());
+
+    Step collectionToSingletonStep = steps.get(1);
+    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+  }
+
+  @Test
+  public void testToSingletonTranslationWithIsmSideInput() throws Exception {
+    // A "change detector" test that makes sure the translation
+    // of getting a PCollectionView<T> does not change
+    // in bad ways during refactor
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    pipeline.apply(Create.of(1))
+        .apply(View.<Integer>asSingleton());
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(5, steps.size());
+
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> toIsmRecordOutputs =
+        (List<Map<String, Object>>) steps.get(3).getProperties().get(PropertyNames.OUTPUT_INFO);
+    assertTrue(
+        Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
+
+    Step collectionToSingletonStep = steps.get(4);
+    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+  }
+
+  @Test
+  public void testToIterableTranslationWithIsmSideInput() throws Exception {
+    // A "change detector" test that makes sure the translation
+    // of getting a PCollectionView<Iterable<T>> does not change
+    // in bad ways during refactor
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+    pipeline.apply(Create.of(1, 2, 3))
+        .apply(View.<Integer>asIterable());
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(3, steps.size());
+
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> toIsmRecordOutputs =
+        (List<Map<String, Object>>) steps.get(1).getProperties().get(PropertyNames.OUTPUT_INFO);
+    assertTrue(
+        Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
+
+
+    Step collectionToSingletonStep = steps.get(2);
+    assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+  }
+
+  @Test
+  public void testStepDisplayData() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+    DataflowPipeline pipeline = DataflowPipeline.create(options);
+
+    DoFn<Integer, Integer> fn1 = new DoFn<Integer, Integer>() {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(c.element());
+      }
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder
+            .add("foo", "bar")
+            .add("foo2", DataflowPipelineTranslatorTest.class)
+                .withLabel("Test Class")
+                .withLinkUrl("http://www.google.com");
+      }
+    };
+
+    DoFn<Integer, Integer> fn2 = new DoFn<Integer, Integer>() {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(c.element());
+      }
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder.add("foo3", 1234);
+      }
+    };
+
+    pipeline
+      .apply(Create.of(1, 2, 3))
+      .apply(ParDo.of(fn1))
+      .apply(ParDo.of(fn2));
+
+    Job job = translator.translate(
+        pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
+
+    List<Step> steps = job.getSteps();
+    assertEquals(3, steps.size());
+
+    Map<String, Object> parDo1Properties = steps.get(1).getProperties();
+    Map<String, Object> parDo2Properties = steps.get(2).getProperties();
+    assertThat(parDo1Properties, hasKey("display_data"));
+
+    Collection<Map<String, String>> fn1displayData =
+            (Collection<Map<String, String>>) parDo1Properties.get("display_data");
+    Collection<Map<String, String>> fn2displayData =
+            (Collection<Map<String, String>>) parDo2Properties.get("display_data");
+
+    ImmutableList expectedFn1DisplayData = ImmutableList.of(
+            ImmutableMap.<String, String>builder()
+              .put("namespace", fn1.getClass().getName())
+              .put("key", "foo")
+              .put("type", "STRING")
+              .put("value", "bar")
+              .build(),
+            ImmutableMap.<String, String>builder()
+              .put("namespace", fn1.getClass().getName())
+              .put("key", "foo2")
+              .put("type", "JAVA_CLASS")
+              .put("value", DataflowPipelineTranslatorTest.class.getName())
+              .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
+              .put("label", "Test Class")
+              .put("linkUrl", "http://www.google.com")
+              .build()
+    );
+
+    ImmutableList expectedFn2DisplayData = ImmutableList.of(
+            ImmutableMap.<String, Object>builder()
+                    .put("namespace", fn2.getClass().getName())
+                    .put("key", "foo3")
+                    .put("type", "INTEGER")
+                    .put("value", 1234L)
+                    .build()
+    );
+
+    assertEquals(expectedFn1DisplayData, fn1displayData);
+    assertEquals(expectedFn2DisplayData, fn2displayData);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
new file mode 100644
index 0000000..20d2bc8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
@@ -0,0 +1,274 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.readFromSource;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Sample;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Duration;
+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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link CustomSources}.
+ */
+@RunWith(JUnit4.class)
+public class CustomSourcesTest {
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+  @Rule public ExpectedLogs logged = ExpectedLogs.none(CustomSources.class);
+
+  static class TestIO {
+    public static Read fromRange(int from, int to) {
+      return new Read(from, to, false);
+    }
+
+    static class Read extends BoundedSource<Integer> {
+      final int from;
+      final int to;
+      final boolean produceTimestamps;
+
+      Read(int from, int to, boolean produceTimestamps) {
+        this.from = from;
+        this.to = to;
+        this.produceTimestamps = produceTimestamps;
+      }
+
+      public Read withTimestampsMillis() {
+        return new Read(from, to, true);
+      }
+
+      @Override
+      public List<Read> splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options)
+          throws Exception {
+        List<Read> res = new ArrayList<>();
+        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+        float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers();
+        for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) {
+          res.add(new Read(
+              Math.round(from + i * step), Math.round(from + (i + 1) * step),
+              produceTimestamps));
+        }
+        return res;
+      }
+
+      @Override
+      public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+        return 8 * (to - from);
+      }
+
+      @Override
+      public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+        return true;
+      }
+
+      @Override
+      public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
+        return new RangeReader(this);
+      }
+
+      @Override
+      public void validate() {}
+
+      @Override
+      public String toString() {
+        return "[" + from + ", " + to + ")";
+      }
+
+      @Override
+      public Coder<Integer> getDefaultOutputCoder() {
+        return BigEndianIntegerCoder.of();
+      }
+
+      private static class RangeReader extends BoundedReader<Integer> {
+        // To verify that BasicSerializableSourceFormat calls our methods according to protocol.
+        enum State {
+          UNSTARTED,
+          STARTED,
+          FINISHED
+        }
+        private Read source;
+        private int current = -1;
+        private State state = State.UNSTARTED;
+
+        public RangeReader(Read source) {
+          this.source = source;
+        }
+
+        @Override
+        public boolean start() throws IOException {
+          Preconditions.checkState(state == State.UNSTARTED);
+          state = State.STARTED;
+          current = source.from;
+          return (current < source.to);
+        }
+
+        @Override
+        public boolean advance() throws IOException {
+          Preconditions.checkState(state == State.STARTED);
+          if (current == source.to - 1) {
+            state = State.FINISHED;
+            return false;
+          }
+          current++;
+          return true;
+        }
+
+        @Override
+        public Integer getCurrent() {
+          Preconditions.checkState(state == State.STARTED);
+          return current;
+        }
+
+        @Override
+        public Instant getCurrentTimestamp() {
+          return source.produceTimestamps
+              ? new Instant(current /* as millis */) : BoundedWindow.TIMESTAMP_MIN_VALUE;
+        }
+
+        @Override
+        public void close() throws IOException {
+          Preconditions.checkState(state == State.STARTED || state == State.FINISHED);
+          state = State.FINISHED;
+        }
+
+        @Override
+        public Read getCurrentSource() {
+          return source;
+        }
+
+        @Override
+        public Read splitAtFraction(double fraction) {
+          int proposedIndex = (int) (source.from + fraction * (source.to - source.from));
+          if (proposedIndex <= current) {
+            return null;
+          }
+          Read primary = new Read(source.from, proposedIndex, source.produceTimestamps);
+          Read residual = new Read(proposedIndex, source.to, source.produceTimestamps);
+          this.source = primary;
+          return residual;
+        }
+
+        @Override
+        public Double getFractionConsumed() {
+          return (current == -1)
+              ? 0.0
+              : (1.0 * (1 + current - source.from) / (source.to - source.from));
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testDirectPipelineWithoutTimestamps() throws Exception {
+    Pipeline p = TestPipeline.create();
+    PCollection<Integer> sum = p
+        .apply(Read.from(TestIO.fromRange(10, 20)))
+        .apply(Sum.integersGlobally())
+        .apply(Sample.<Integer>any(1));
+
+    PAssert.thatSingleton(sum).isEqualTo(145);
+    p.run();
+  }
+
+  @Test
+  public void testDirectPipelineWithTimestamps() throws Exception {
+    Pipeline p = TestPipeline.create();
+    PCollection<Integer> sums =
+        p.apply(Read.from(TestIO.fromRange(10, 20).withTimestampsMillis()))
+         .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(3))))
+         .apply(Sum.integersGlobally().withoutDefaults());
+    // Should group into [10 11] [12 13 14] [15 16 17] [18 19].
+    PAssert.that(sums).containsInAnyOrder(21, 37, 39, 48);
+    p.run();
+  }
+
+  @Test
+  public void testRangeProgressAndSplitAtFraction() throws Exception {
+    // Show basic usage of getFractionConsumed and splitAtFraction.
+    // This test only tests TestIO itself, not BasicSerializableSourceFormat.
+
+    DataflowPipelineOptions options =
+        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+    TestIO.Read source = TestIO.fromRange(10, 20);
+    try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
+      assertEquals(0, reader.getFractionConsumed().intValue());
+      assertTrue(reader.start());
+      assertEquals(0.1, reader.getFractionConsumed(), 1e-6);
+      assertTrue(reader.advance());
+      assertEquals(0.2, reader.getFractionConsumed(), 1e-6);
+      // Already past 0.0 and 0.1.
+      assertNull(reader.splitAtFraction(0.0));
+      assertNull(reader.splitAtFraction(0.1));
+
+      {
+        TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.5);
+        assertNotNull(residual);
+        TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
+        assertThat(readFromSource(primary, options), contains(10, 11, 12, 13, 14));
+        assertThat(readFromSource(residual, options), contains(15, 16, 17, 18, 19));
+      }
+
+      // Range is now [10, 15) and we are at 12.
+      {
+        TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.8); // give up 14.
+        assertNotNull(residual);
+        TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
+        assertThat(readFromSource(primary, options), contains(10, 11, 12, 13));
+        assertThat(readFromSource(residual, options), contains(14));
+      }
+
+      assertTrue(reader.advance());
+      assertEquals(12, reader.getCurrent().intValue());
+      assertTrue(reader.advance());
+      assertEquals(13, reader.getCurrent().intValue());
+      assertFalse(reader.advance());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
new file mode 100644
index 0000000..0a78a6d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
@@ -0,0 +1,377 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.google.cloud.dataflow.sdk.testing;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.Json;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+import com.google.cloud.dataflow.sdk.util.TimeUtil;
+import com.google.cloud.dataflow.sdk.util.Transport;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+import org.joda.time.Instant;
+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.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+/** Tests for {@link TestDataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class TestDataflowPipelineRunnerTest {
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+  @Mock private MockHttpTransport transport;
+  @Mock private MockLowLevelHttpRequest request;
+  @Mock private GcsUtil mockGcsUtil;
+
+  private TestDataflowPipelineOptions options;
+  private Dataflow service;
+
+  @Before
+  public void setUp() throws Exception {
+    MockitoAnnotations.initMocks(this);
+    when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
+    doCallRealMethod().when(request).getContentAsString();
+    service = new Dataflow(transport, Transport.getJsonFactory(), null);
+
+    options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+    options.setAppName("TestAppName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setDataflowClient(service);
+    options.setRunner(TestDataflowPipelineRunner.class);
+    options.setPathValidatorClass(NoopPathValidator.class);
+  }
+
+  @Test
+  public void testToString() {
+    assertEquals("TestDataflowPipelineRunner#TestAppName",
+        new TestDataflowPipelineRunner(options).toString());
+  }
+
+  @Test
+  public void testRunBatchJobThatSucceeds() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    assertEquals(mockJob, runner.run(p, mockRunner));
+  }
+
+  @Test
+  public void testRunBatchJobThatFails() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testBatchPipelineFailsIfException() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenAnswer(new Answer<State>() {
+          @Override
+          public State answer(InvocationOnMock invocation) {
+            JobMessage message = new JobMessage();
+            message.setMessageText("FooException");
+            message.setTime(TimeUtil.toCloudTime(Instant.now()));
+            message.setMessageImportance("JOB_MESSAGE_ERROR");
+            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+                .process(Arrays.asList(message));
+            return State.CANCELLED;
+          }
+        });
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      assertThat(expected.getMessage(), containsString("FooException"));
+      verify(mockJob, atLeastOnce()).cancel();
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testRunStreamingJobThatSucceeds() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testRunStreamingJobThatFails() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    doReturn(State.DONE).when(job).getState();
+    assertEquals(Optional.of(true), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testCheckingForSuccessWhenPAssertFails() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    doReturn(State.DONE).when(job).getState();
+    assertEquals(Optional.of(false), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, false /* tentative */));
+    doReturn(State.RUNNING).when(job).getState();
+    assertEquals(Optional.absent(), runner.checkForSuccess(job));
+  }
+
+  private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
+      throws Exception {
+    MetricStructuredName name = new MetricStructuredName();
+    name.setName(success ? "PAssertSuccess" : "PAssertFailure");
+    name.setContext(
+        tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
+
+    MetricUpdate metric = new MetricUpdate();
+    metric.setName(name);
+    metric.setScalar(BigDecimal.ONE);
+
+    MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
+    response.setContentType(Json.MEDIA_TYPE);
+    JobMetrics jobMetrics = new JobMetrics();
+    jobMetrics.setMetrics(Lists.newArrayList(metric));
+    // N.B. Setting the factory is necessary in order to get valid JSON.
+    jobMetrics.setFactory(Transport.getJsonFactory());
+    response.setContent(jobMetrics.toPrettyString());
+    return response;
+  }
+
+  @Test
+  public void testStreamingPipelineFailsIfServiceFails() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, false /* tentative */));
+    doReturn(State.FAILED).when(job).getState();
+    assertEquals(Optional.of(false), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testStreamingPipelineFailsIfException() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenAnswer(new Answer<State>() {
+          @Override
+          public State answer(InvocationOnMock invocation) {
+            JobMessage message = new JobMessage();
+            message.setMessageText("FooException");
+            message.setTime(TimeUtil.toCloudTime(Instant.now()));
+            message.setMessageImportance("JOB_MESSAGE_ERROR");
+            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+                .process(Arrays.asList(message));
+            return State.CANCELLED;
+          }
+        });
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      assertThat(expected.getMessage(), containsString("FooException"));
+      verify(mockJob, atLeastOnce()).cancel();
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
new file mode 100644
index 0000000..914a484
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.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 com.google.cloud.dataflow.sdk.transforms;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/** Tests for {@link GroupByKey} for the {@link DataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class DataflowGroupByKeyTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  /**
+   * Create a test pipeline that uses the {@link DataflowPipelineRunner} so that {@link GroupByKey}
+   * is not expanded. This is used for verifying that even without expansion the proper errors show
+   * up.
+   */
+  private Pipeline createTestServiceRunner() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("someproject");
+    options.setStagingLocation("gs://staging");
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setDataflowClient(null);
+    return Pipeline.create(options);
+  }
+
+  @Test
+  public void testInvalidWindowsService() {
+    Pipeline p = createTestServiceRunner();
+
+    List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
+
+    PCollection<KV<String, Integer>> input =
+        p.apply(Create.of(ungroupedPairs)
+            .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
+        .apply(Window.<KV<String, Integer>>into(
+            Sessions.withGapDuration(Duration.standardMinutes(1))));
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("GroupByKey must have a valid Window merge function");
+    input
+        .apply("GroupByKey", GroupByKey.<String, Integer>create())
+        .apply("GroupByKeyAgain", GroupByKey.<String, Iterable<Integer>>create());
+  }
+
+  @Test
+  public void testGroupByKeyServiceUnbounded() {
+    Pipeline p = createTestServiceRunner();
+
+    PCollection<KV<String, Integer>> input =
+        p.apply(
+            new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
+              @Override
+              public PCollection<KV<String, Integer>> apply(PBegin input) {
+                return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
+                        input.getPipeline(),
+                        WindowingStrategy.globalDefault(),
+                        PCollection.IsBounded.UNBOUNDED)
+                    .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
+              }
+            });
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(
+        "GroupByKey cannot be applied to non-bounded PCollection in the GlobalWindow without "
+        + "a trigger. Use a Window.into or Window.triggering transform prior to GroupByKey.");
+
+    input.apply("GroupByKey", GroupByKey.<String, Integer>create());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
new file mode 100644
index 0000000..936b7c6
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.transforms;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link View} for a {@link DataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class DataflowViewTest {
+  @Rule
+  public transient ExpectedException thrown = ExpectedException.none();
+
+  private Pipeline createTestBatchRunner() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("someproject");
+    options.setStagingLocation("gs://staging");
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setDataflowClient(null);
+    return Pipeline.create(options);
+  }
+
+  private Pipeline createTestStreamingRunner() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setStreaming(true);
+    options.setProject("someproject");
+    options.setStagingLocation("gs://staging");
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setDataflowClient(null);
+    return Pipeline.create(options);
+  }
+
+  private void testViewUnbounded(
+      Pipeline pipeline,
+      PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("Unable to create a side-input view from input");
+    thrown.expectCause(
+        ThrowableMessageMatcher.hasMessage(Matchers.containsString("non-bounded PCollection")));
+    pipeline
+        .apply(
+            new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
+              @Override
+              public PCollection<KV<String, Integer>> apply(PBegin input) {
+                return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
+                        input.getPipeline(),
+                        WindowingStrategy.globalDefault(),
+                        PCollection.IsBounded.UNBOUNDED)
+                    .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
+              }
+            })
+        .apply(view);
+  }
+
+  private void testViewNonmerging(
+      Pipeline pipeline,
+      PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("Unable to create a side-input view from input");
+    thrown.expectCause(
+        ThrowableMessageMatcher.hasMessage(Matchers.containsString("Consumed by GroupByKey")));
+    pipeline.apply(Create.<KV<String, Integer>>of(KV.of("hello", 5)))
+        .apply(Window.<KV<String, Integer>>into(new InvalidWindows<>(
+            "Consumed by GroupByKey", FixedWindows.of(Duration.standardHours(1)))))
+        .apply(view);
+  }
+
+  @Test
+  public void testViewUnboundedAsSingletonBatch() {
+    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
+  }
+
+  @Test
+  public void testViewUnboundedAsSingletonStreaming() {
+    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
+  }
+
+  @Test
+  public void testViewUnboundedAsIterableBatch() {
+    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
+  }
+
+  @Test
+  public void testViewUnboundedAsIterableStreaming() {
+    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
+  }
+
+  @Test
+  public void testViewUnboundedAsListBatch() {
+    testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asList());
+  }
+
+  @Test
+  public void testViewUnboundedAsListStreaming() {
+    testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
+  }
+
+  @Test
+  public void testViewUnboundedAsMapBatch() {
+    testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMap());
+  }
+
+  @Test
+  public void testViewUnboundedAsMapStreaming() {
+    testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMap());
+  }
+
+  @Test
+  public void testViewUnboundedAsMultimapBatch() {
+    testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMultimap());
+  }
+
+  @Test
+  public void testViewUnboundedAsMultimapStreaming() {
+    testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMultimap());
+  }
+
+  @Test
+  public void testViewNonmergingAsSingletonBatch() {
+    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
+  }
+
+  @Test
+  public void testViewNonmergingAsSingletonStreaming() {
+    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
+  }
+
+  @Test
+  public void testViewNonmergingAsIterableBatch() {
+    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
+  }
+
+  @Test
+  public void testViewNonmergingAsIterableStreaming() {
+    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
+  }
+
+  @Test
+  public void testViewNonmergingAsListBatch() {
+    testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asList());
+  }
+
+  @Test
+  public void testViewNonmergingAsListStreaming() {
+    testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
+  }
+
+  @Test
+  public void testViewNonmergingAsMapBatch() {
+    testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMap());
+  }
+
+  @Test
+  public void testViewNonmergingAsMapStreaming() {
+    testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMap());
+  }
+
+  @Test
+  public void testViewNonmergingAsMultimapBatch() {
+    testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMultimap());
+  }
+
+  @Test
+  public void testViewNonmergingAsMultimapStreaming() {
+    testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMultimap());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
new file mode 100644
index 0000000..7f35fd3
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+
+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;
+
+/** Tests for {@link DataflowPathValidator}. */
+@RunWith(JUnit4.class)
+public class DataflowPathValidatorTest {
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+
+  @Mock private GcsUtil mockGcsUtil;
+  private DataflowPathValidator validator;
+
+  @Before
+  public void setUp() throws Exception {
+    MockitoAnnotations.initMocks(this);
+    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
+    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setGcpCredential(new TestCredential());
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setGcsUtil(mockGcsUtil);
+    validator = new DataflowPathValidator(options);
+  }
+
+  @Test
+  public void testValidFilePattern() {
+    validator.validateInputFilePatternSupported("gs://bucket/path");
+  }
+
+  @Test
+  public void testInvalidFilePattern() {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+    validator.validateInputFilePatternSupported("/local/path");
+  }
+
+  @Test
+  public void testWhenBucketDoesNotExist() throws Exception {
+    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(false);
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Could not find file gs://non-existent-bucket/location");
+    validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
+  }
+
+  @Test
+  public void testValidOutputPrefix() {
+    validator.validateOutputFilePrefixSupported("gs://bucket/path");
+  }
+
+  @Test
+  public void testInvalidOutputPrefix() {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+    validator.validateOutputFilePrefixSupported("/local/path");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
new file mode 100644
index 0000000..23e12de
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.ListJobMessagesResponse;
+import com.google.cloud.dataflow.sdk.PipelineResult.State;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+
+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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for MonitoringUtil.
+ */
+@RunWith(JUnit4.class)
+public class MonitoringUtilTest {
+  private static final String PROJECT_ID = "someProject";
+  private static final String JOB_ID = "1234";
+
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testGetJobMessages() throws IOException {
+    Dataflow.Projects.Jobs.Messages mockMessages = mock(Dataflow.Projects.Jobs.Messages.class);
+
+    // Two requests are needed to get all the messages.
+    Dataflow.Projects.Jobs.Messages.List firstRequest =
+        mock(Dataflow.Projects.Jobs.Messages.List.class);
+    Dataflow.Projects.Jobs.Messages.List secondRequest =
+        mock(Dataflow.Projects.Jobs.Messages.List.class);
+
+    when(mockMessages.list(PROJECT_ID, JOB_ID)).thenReturn(firstRequest).thenReturn(secondRequest);
+
+    ListJobMessagesResponse firstResponse = new ListJobMessagesResponse();
+    firstResponse.setJobMessages(new ArrayList<JobMessage>());
+    for (int i = 0; i < 100; ++i) {
+      JobMessage message = new JobMessage();
+      message.setId("message_" + i);
+      message.setTime(TimeUtil.toCloudTime(new Instant(i)));
+      firstResponse.getJobMessages().add(message);
+    }
+    String pageToken = "page_token";
+    firstResponse.setNextPageToken(pageToken);
+
+    ListJobMessagesResponse secondResponse = new ListJobMessagesResponse();
+    secondResponse.setJobMessages(new ArrayList<JobMessage>());
+    for (int i = 100; i < 150; ++i) {
+      JobMessage message = new JobMessage();
+      message.setId("message_" + i);
+      message.setTime(TimeUtil.toCloudTime(new Instant(i)));
+      secondResponse.getJobMessages().add(message);
+    }
+
+    when(firstRequest.execute()).thenReturn(firstResponse);
+    when(secondRequest.execute()).thenReturn(secondResponse);
+
+    MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages);
+
+    List<JobMessage> messages = util.getJobMessages(JOB_ID, -1);
+
+    verify(secondRequest).setPageToken(pageToken);
+
+    assertEquals(150, messages.size());
+  }
+
+  @Test
+  public void testToStateCreatesState() {
+    String stateName = "JOB_STATE_DONE";
+
+    State result = MonitoringUtil.toState(stateName);
+
+    assertEquals(State.DONE, result);
+  }
+
+  @Test
+  public void testToStateWithNullReturnsUnknown() {
+    String stateName = null;
+
+    State result = MonitoringUtil.toState(stateName);
+
+    assertEquals(State.UNKNOWN, result);
+  }
+
+  @Test
+  public void testToStateWithOtherValueReturnsUnknown() {
+    String stateName = "FOO_BAR_BAZ";
+
+    State result = MonitoringUtil.toState(stateName);
+
+    assertEquals(State.UNKNOWN, result);
+  }
+
+  @Test
+  public void testDontOverrideEndpointWithDefaultApi() {
+    DataflowPipelineOptions options =
+        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+    options.setProject(PROJECT_ID);
+    options.setGcpCredential(new TestCredential());
+    String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
+    assertEquals("gcloud alpha dataflow jobs --project=someProject cancel 1234", cancelCommand);
+  }
+
+  @Test
+  public void testOverridesEndpointWithStagedDataflowEndpoint() {
+    DataflowPipelineOptions options =
+        PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+    options.setProject(PROJECT_ID);
+    options.setGcpCredential(new TestCredential());
+    String stagingDataflowEndpoint = "v0neverExisted";
+    options.setDataflowEndpoint(stagingDataflowEndpoint);
+    String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
+    assertEquals(
+        "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW=https://dataflow.googleapis.com/v0neverExisted/ "
+        + "gcloud alpha dataflow jobs --project=someProject cancel 1234",
+        cancelCommand);
+  }
+}



[10/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
deleted file mode 100644
index 2ca845d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
+++ /dev/null
@@ -1,264 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.GoogleApiDebugOptions.GoogleApiTracer;
-import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Deserializer;
-import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Serializer;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.Context;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
-
-import java.lang.reflect.Proxy;
-import java.util.ServiceLoader;
-
-import javax.annotation.concurrent.ThreadSafe;
-
-/**
- * PipelineOptions are used to configure Pipelines. You can extend {@link PipelineOptions}
- * to create custom configuration options specific to your {@link Pipeline},
- * for both local execution and execution via a {@link PipelineRunner}.
- *
- * <p>{@link PipelineOptions} and their subinterfaces represent a collection of properties
- * which can be manipulated in a type safe manner. {@link PipelineOptions} is backed by a
- * dynamic {@link Proxy} which allows for type safe manipulation of properties in an extensible
- * fashion through plain old Java interfaces.
- *
- * <p>{@link PipelineOptions} can be created with {@link PipelineOptionsFactory#create()}
- * and {@link PipelineOptionsFactory#as(Class)}. They can be created
- * from command-line arguments with {@link PipelineOptionsFactory#fromArgs(String[])}.
- * They can be converted to another type by invoking {@link PipelineOptions#as(Class)} and
- * can be accessed from within a {@link DoFn} by invoking
- * {@link Context#getPipelineOptions()}.
- *
- * <p>For example:
- * <pre>{@code
- * // The most common way to construct PipelineOptions is via command-line argument parsing:
- * public static void main(String[] args) {
- *   // Will parse the arguments passed into the application and construct a PipelineOptions
- *   // Note that --help will print registered options, and --help=PipelineOptionsClassName
- *   // will print out usage for the specific class.
- *   PipelineOptions options =
- *       PipelineOptionsFactory.fromArgs(args).create();
- *
- *   Pipeline p = Pipeline.create(options);
- *   ...
- *   p.run();
- * }
- *
- * // To create options for the DirectPipeline:
- * DirectPipelineOptions directPipelineOptions =
- *     PipelineOptionsFactory.as(DirectPipelineOptions.class);
- * directPipelineOptions.setStreaming(true);
- *
- * // To cast from one type to another using the as(Class) method:
- * DataflowPipelineOptions dataflowPipelineOptions =
- *     directPipelineOptions.as(DataflowPipelineOptions.class);
- *
- * // Options for the same property are shared between types
- * // The statement below will print out "true"
- * System.out.println(dataflowPipelineOptions.isStreaming());
- *
- * // Prints out registered options.
- * PipelineOptionsFactory.printHelp(System.out);
- *
- * // Prints out options which are available to be set on DataflowPipelineOptions
- * PipelineOptionsFactory.printHelp(System.out, DataflowPipelineOptions.class);
- * }</pre>
- *
- * <h2>Defining Your Own PipelineOptions</h2>
- *
- * Defining your own {@link PipelineOptions} is the way for you to make configuration
- * options available for both local execution and execution via a {@link PipelineRunner}.
- * By having PipelineOptionsFactory as your command-line interpreter, you will provide
- * a standardized way for users to interact with your application via the command-line.
- *
- * <p>To define your own {@link PipelineOptions}, you create an interface which
- * extends {@link PipelineOptions} and define getter/setter pairs. These
- * getter/setter pairs define a collection of
- * <a href="https://docs.oracle.com/javase/tutorial/javabeans/writing/properties.html">
- * JavaBean properties</a>.
- *
- * <p>For example:
- * <pre>{@code
- *  // Creates a user defined property called "myProperty"
- *  public interface MyOptions extends PipelineOptions {
- *    String getMyProperty();
- *    void setMyProperty(String value);
- *  }
- * }</pre>
- *
- * <p>Note: Please see the section on Registration below when using custom property types.
- *
- * <h3>Restrictions</h3>
- *
- * Since PipelineOptions can be "cast" to multiple types dynamically using
- * {@link PipelineOptions#as(Class)}, a property must conform to the following set of restrictions:
- * <ul>
- *   <li>Any property with the same name must have the same return type for all derived
- *       interfaces of {@link PipelineOptions}.
- *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
- *       getter and setter method.
- *   <li>Every method must conform to being a getter or setter for a JavaBean.
- *   <li>The derived interface of {@link PipelineOptions} must be composable with every interface
- *       part registered with the PipelineOptionsFactory.
- *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
- *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for
- *       this property must be annotated with {@link JsonIgnore @JsonIgnore}.
- * </ul>
- *
- * <h3>Annotations For PipelineOptions</h3>
- *
- * {@link Description @Description} can be used to annotate an interface or a getter
- * with useful information which is output when {@code --help}
- * is invoked via {@link PipelineOptionsFactory#fromArgs(String[])}.
- *
- * <p>{@link Default @Default} represents a set of annotations that can be used to annotate getter
- * properties on {@link PipelineOptions} with information representing the default value to be
- * returned if no value is specified. Any default implementation (using the {@code default} keyword)
- * is ignored.
- *
- * <p>{@link Hidden @Hidden} hides an option from being listed when {@code --help}
- * is invoked via {@link PipelineOptionsFactory#fromArgs(String[])}.
- *
- * <p>{@link Validation @Validation} represents a set of annotations that can be used to annotate
- * getter properties on {@link PipelineOptions} with information representing the validation
- * criteria to be used when validating with the {@link PipelineOptionsValidator}. Validation
- * will be performed if during construction of the {@link PipelineOptions},
- * {@link PipelineOptionsFactory#withValidation()} is invoked.
- *
- * <p>{@link JsonIgnore @JsonIgnore} is used to prevent a property from being serialized and
- * available during execution of {@link DoFn}. See the Serialization section below for more
- * details.
- *
- * <h2>Registration Of PipelineOptions</h2>
- *
- * Registration of {@link PipelineOptions} by an application guarantees that the
- * {@link PipelineOptions} is composable during execution of their {@link Pipeline} and
- * meets the restrictions listed above or will fail during registration. Registration
- * also lists the registered {@link PipelineOptions} when {@code --help}
- * is invoked via {@link PipelineOptionsFactory#fromArgs(String[])}.
- *
- * <p>Registration can be performed by invoking {@link PipelineOptionsFactory#register} within
- * a users application or via automatic registration by creating a {@link ServiceLoader} entry
- * and a concrete implementation of the {@link PipelineOptionsRegistrar} interface.
- *
- * <p>It is optional but recommended to use one of the many build time tools such as
- * {@link AutoService} to generate the necessary META-INF files automatically.
- *
- * <p>A list of registered options can be fetched from
- * {@link PipelineOptionsFactory#getRegisteredOptions()}.
- *
- * <h2>Serialization Of PipelineOptions</h2>
- *
- * {@link PipelineRunner}s require support for options to be serialized. Each property
- * within {@link PipelineOptions} must be able to be serialized using Jackson's
- * {@link ObjectMapper} or the getter method for the property annotated with
- * {@link JsonIgnore @JsonIgnore}.
- *
- * <p>Jackson supports serialization of many types and supports a useful set of
- * <a href="https://github.com/FasterXML/jackson-annotations">annotations</a> to aid in
- * serialization of custom types. We point you to the public
- * <a href="https://github.com/FasterXML/jackson">Jackson documentation</a> when attempting
- * to add serialization support for your custom types. See {@link GoogleApiTracer} for an
- * example using the Jackson annotations to serialize and deserialize a custom type.
- *
- * <p>Note: It is an error to have the same property available in multiple interfaces with only
- * some of them being annotated with {@link JsonIgnore @JsonIgnore}. It is also an error to mark a
- * setter for a property with {@link JsonIgnore @JsonIgnore}.
- */
-@JsonSerialize(using = Serializer.class)
-@JsonDeserialize(using = Deserializer.class)
-@ThreadSafe
-public interface PipelineOptions {
-  /**
-   * Transforms this object into an object of type {@code <T>} saving each property
-   * that has been manipulated. {@code <T>} must extend {@link PipelineOptions}.
-   *
-   * <p>If {@code <T>} is not registered with the {@link PipelineOptionsFactory}, then we
-   * attempt to verify that {@code <T>} is composable with every interface that this
-   * instance of the {@code PipelineOptions} has seen.
-   *
-   * @param kls The class of the type to transform to.
-   * @return An object of type kls.
-   */
-  <T extends PipelineOptions> T as(Class<T> kls);
-
-  /**
-   * Makes a deep clone of this object, and transforms the cloned object into the specified
-   * type {@code kls}. See {@link #as} for more information about the conversion.
-   *
-   * <p>Properties that are marked with {@code @JsonIgnore} will not be cloned.
-   */
-  <T extends PipelineOptions> T cloneAs(Class<T> kls);
-
-  /**
-   * The pipeline runner that will be used to execute the pipeline.
-   * For registered runners, the class name can be specified, otherwise the fully
-   * qualified name needs to be specified.
-   */
-  @Validation.Required
-  @Description("The pipeline runner that will be used to execute the pipeline. "
-      + "For registered runners, the class name can be specified, otherwise the fully "
-      + "qualified name needs to be specified.")
-  @Default.Class(DirectPipelineRunner.class)
-  Class<? extends PipelineRunner<?>> getRunner();
-  void setRunner(Class<? extends PipelineRunner<?>> kls);
-
-  /**
-   * Enumeration of the possible states for a given check.
-   */
-  public static enum CheckEnabled {
-    OFF,
-    WARNING,
-    ERROR;
-  }
-
-  /**
-   * Whether to check for stable unique names on each transform. This is necessary to
-   * support updating of pipelines.
-   */
-  @Validation.Required
-  @Description("Whether to check for stable unique names on each transform. This is necessary to "
-      + "support updating of pipelines.")
-  @Default.Enum("WARNING")
-  CheckEnabled getStableUniqueNames();
-  void setStableUniqueNames(CheckEnabled enabled);
-
-  /**
-   * A pipeline level default location for storing temporary files.
-   *
-   * <p>This can be a path of any file system.
-   *
-   * <p>{@link #getTempLocation()} can be used as a default location in other
-   * {@link PipelineOptions}.
-   *
-   * <p>If it is unset, {@link PipelineRunner} can override it.
-   */
-  @Description("A pipeline level default location for storing temporary files.")
-  String getTempLocation();
-  void setTempLocation(String value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
deleted file mode 100644
index dac7726..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
+++ /dev/null
@@ -1,1490 +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 com.google.cloud.dataflow.sdk.options;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
-import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers;
-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.base.Throwables;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableListMultimap;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.collect.SortedSetMultimap;
-import com.google.common.collect.TreeMultimap;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.databind.JavaType;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.beans.BeanInfo;
-import java.beans.IntrospectionException;
-import java.beans.Introspector;
-import java.beans.PropertyDescriptor;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.annotation.Annotation;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.Proxy;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import javax.annotation.Nullable;
-
-/**
- * Constructs a {@link PipelineOptions} or any derived interface that is composable to any other
- * derived interface of {@link PipelineOptions} via the {@link PipelineOptions#as} method. Being
- * able to compose one derived interface of {@link PipelineOptions} to another has the following
- * restrictions:
- * <ul>
- *   <li>Any property with the same name must have the same return type for all derived interfaces
- *       of {@link PipelineOptions}.
- *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
- *       getter and setter method.
- *   <li>Every method must conform to being a getter or setter for a JavaBean.
- *   <li>The derived interface of {@link PipelineOptions} must be composable with every interface
- *       registered with this factory.
- * </ul>
- *
- * <p>See the <a
- * href="http://www.oracle.com/technetwork/java/javase/documentation/spec-136004.html">JavaBeans
- * specification</a> for more details as to what constitutes a property.
- */
-public class PipelineOptionsFactory {
-  /**
-   * Creates and returns an object that implements {@link PipelineOptions}.
-   * This sets the {@link ApplicationNameOptions#getAppName() "appName"} to the calling
-   * {@link Class#getSimpleName() classes simple name}.
-   *
-   * @return An object that implements {@link PipelineOptions}.
-   */
-  public static PipelineOptions create() {
-    return new Builder().as(PipelineOptions.class);
-  }
-
-  /**
-   * Creates and returns an object that implements {@code <T>}.
-   * This sets the {@link ApplicationNameOptions#getAppName() "appName"} to the calling
-   * {@link Class#getSimpleName() classes simple name}.
-   *
-   * <p>Note that {@code <T>} must be composable with every registered interface with this factory.
-   * See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more details.
-   *
-   * @return An object that implements {@code <T>}.
-   */
-  public static <T extends PipelineOptions> T as(Class<T> klass) {
-    return new Builder().as(klass);
-  }
-
-  /**
-   * Sets the command line arguments to parse when constructing the {@link PipelineOptions}.
-   *
-   * <p>Example GNU style command line arguments:
-   * <pre>
-   *   --project=MyProject (simple property, will set the "project" property to "MyProject")
-   *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
-   *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
-   *   --x=1 --x=2 --x=3 (list style simple property, will set the "x" property to [1, 2, 3])
-   *   --x=1,2,3 (shorthand list style simple property, will set the "x" property to [1, 2, 3])
-   *   --complexObject='{"key1":"value1",...} (JSON format for all other complex types)
-   * </pre>
-   *
-   * <p>Simple properties are able to bound to {@link String}, {@link Class}, enums and Java
-   * primitives {@code boolean}, {@code byte}, {@code short}, {@code int}, {@code long},
-   * {@code float}, {@code double} and their primitive wrapper classes.
-   *
-   * <p>Simple list style properties are able to be bound to {@code boolean[]}, {@code char[]},
-   * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]},
-   * {@code Class[]}, enum arrays, {@code String[]}, and {@code List<String>}.
-   *
-   * <p>JSON format is required for all other types.
-   *
-   * <p>By default, strict parsing is enabled and arguments must conform to be either
-   * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with
-   * {@link Builder#withoutStrictParsing()}. Empty or null arguments will be ignored whether
-   * or not strict parsing is enabled.
-   *
-   * <p>Help information can be output to {@link System#out} by specifying {@code --help} as an
-   * argument. After help is printed, the application will exit. Specifying only {@code --help}
-   * will print out the list of
-   * {@link PipelineOptionsFactory#getRegisteredOptions() registered options}
-   * by invoking {@link PipelineOptionsFactory#printHelp(PrintStream)}. Specifying
-   * {@code --help=PipelineOptionsClassName} will print out detailed usage information about the
-   * specifically requested PipelineOptions by invoking
-   * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)}.
-   */
-  public static Builder fromArgs(String[] args) {
-    return new Builder().fromArgs(args);
-  }
-
-  /**
-   * After creation we will validate that {@code <T>} conforms to all the
-   * validation criteria. See
-   * {@link PipelineOptionsValidator#validate(Class, PipelineOptions)} for more details about
-   * validation.
-   */
-  public Builder withValidation() {
-    return new Builder().withValidation();
-  }
-
-  /** A fluent {@link PipelineOptions} builder. */
-  public static class Builder {
-    private final String defaultAppName;
-    private final String[] args;
-    private final boolean validation;
-    private final boolean strictParsing;
-
-    // Do not allow direct instantiation
-    private Builder() {
-      this(null, false, true);
-    }
-
-    private Builder(String[] args, boolean validation,
-        boolean strictParsing) {
-      this.defaultAppName = findCallersClassName();
-      this.args = args;
-      this.validation = validation;
-      this.strictParsing = strictParsing;
-    }
-
-    /**
-     * Sets the command line arguments to parse when constructing the {@link PipelineOptions}.
-     *
-     * <p>Example GNU style command line arguments:
-     * <pre>
-     *   --project=MyProject (simple property, will set the "project" property to "MyProject")
-     *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
-     *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
-     *   --x=1 --x=2 --x=3 (list style simple property, will set the "x" property to [1, 2, 3])
-     *   --x=1,2,3 (shorthand list style simple property, will set the "x" property to [1, 2, 3])
-     *   --complexObject='{"key1":"value1",...} (JSON format for all other complex types)
-     * </pre>
-     *
-     * <p>Simple properties are able to bound to {@link String}, {@link Class}, enums and Java
-     * primitives {@code boolean}, {@code byte}, {@code short}, {@code int}, {@code long},
-     * {@code float}, {@code double} and their primitive wrapper classes.
-     *
-     * <p>Simple list style properties are able to be bound to {@code boolean[]}, {@code char[]},
-     * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]},
-     * {@code Class[]}, enum arrays, {@code String[]}, and {@code List<String>}.
-     *
-     * <p>JSON format is required for all other types.
-     *
-     * <p>By default, strict parsing is enabled and arguments must conform to be either
-     * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with
-     * {@link Builder#withoutStrictParsing()}. Empty or null arguments will be ignored whether
-     * or not strict parsing is enabled.
-     *
-     * <p>Help information can be output to {@link System#out} by specifying {@code --help} as an
-     * argument. After help is printed, the application will exit. Specifying only {@code --help}
-     * will print out the list of
-     * {@link PipelineOptionsFactory#getRegisteredOptions() registered options}
-     * by invoking {@link PipelineOptionsFactory#printHelp(PrintStream)}. Specifying
-     * {@code --help=PipelineOptionsClassName} will print out detailed usage information about the
-     * specifically requested PipelineOptions by invoking
-     * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)}.
-     */
-    public Builder fromArgs(String[] args) {
-      Preconditions.checkNotNull(args, "Arguments should not be null.");
-      return new Builder(args, validation, strictParsing);
-    }
-
-    /**
-     * After creation we will validate that {@link PipelineOptions} conforms to all the
-     * validation criteria from {@code <T>}. See
-     * {@link PipelineOptionsValidator#validate(Class, PipelineOptions)} for more details about
-     * validation.
-     */
-    public Builder withValidation() {
-      return new Builder(args, true, strictParsing);
-    }
-
-    /**
-     * During parsing of the arguments, we will skip over improperly formatted and unknown
-     * arguments.
-     */
-    public Builder withoutStrictParsing() {
-      return new Builder(args, validation, false);
-    }
-
-    /**
-     * Creates and returns an object that implements {@link PipelineOptions} using the values
-     * configured on this builder during construction.
-     *
-     * @return An object that implements {@link PipelineOptions}.
-     */
-    public PipelineOptions create() {
-      return as(PipelineOptions.class);
-    }
-
-    /**
-     * Creates and returns an object that implements {@code <T>} using the values configured on
-     * this builder during construction.
-     *
-     * <p>Note that {@code <T>} must be composable with every registered interface with this
-     * factory. See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more
-     * details.
-     *
-     * @return An object that implements {@code <T>}.
-     */
-    public <T extends PipelineOptions> T as(Class<T> klass) {
-      Map<String, Object> initialOptions = Maps.newHashMap();
-
-      // Attempt to parse the arguments into the set of initial options to use
-      if (args != null) {
-        ListMultimap<String, String> options = parseCommandLine(args, strictParsing);
-        LOG.debug("Provided Arguments: {}", options);
-        printHelpUsageAndExitIfNeeded(options, System.out, true /* exit */);
-        initialOptions = parseObjects(klass, options, strictParsing);
-      }
-
-      // Create our proxy
-      ProxyInvocationHandler handler = new ProxyInvocationHandler(initialOptions);
-      T t = handler.as(klass);
-
-      // Set the application name to the default if none was set.
-      ApplicationNameOptions appNameOptions = t.as(ApplicationNameOptions.class);
-      if (appNameOptions.getAppName() == null) {
-        appNameOptions.setAppName(defaultAppName);
-      }
-
-      if (validation) {
-        PipelineOptionsValidator.validate(klass, t);
-      }
-      return t;
-    }
-  }
-
-  /**
-   * Determines whether the generic {@code --help} was requested or help was
-   * requested for a specific class and invokes the appropriate
-   * {@link PipelineOptionsFactory#printHelp(PrintStream)} and
-   * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)} variant.
-   * Prints to the specified {@link PrintStream}, and exits if requested.
-   *
-   * <p>Visible for testing.
-   * {@code printStream} and {@code exit} used for testing.
-   */
-  @SuppressWarnings("unchecked")
-  static boolean printHelpUsageAndExitIfNeeded(ListMultimap<String, String> options,
-      PrintStream printStream, boolean exit) {
-    if (options.containsKey("help")) {
-      final String helpOption = Iterables.getOnlyElement(options.get("help"));
-
-      // Print the generic help if only --help was specified.
-      if (Boolean.TRUE.toString().equals(helpOption)) {
-        printHelp(printStream);
-        if (exit) {
-          System.exit(0);
-        } else {
-          return true;
-        }
-      }
-
-      // Otherwise attempt to print the specific help option.
-      try {
-        Class<?> klass = Class.forName(helpOption);
-        if (!PipelineOptions.class.isAssignableFrom(klass)) {
-          throw new ClassNotFoundException("PipelineOptions of type " + klass + " not found.");
-        }
-        printHelp(printStream, (Class<? extends PipelineOptions>) klass);
-      } catch (ClassNotFoundException e) {
-        // If we didn't find an exact match, look for any that match the class name.
-        Iterable<Class<? extends PipelineOptions>> matches = Iterables.filter(
-            getRegisteredOptions(),
-            new Predicate<Class<? extends PipelineOptions>>() {
-              @Override
-              public boolean apply(Class<? extends PipelineOptions> input) {
-                if (helpOption.contains(".")) {
-                  return input.getName().endsWith(helpOption);
-                } else {
-                  return input.getSimpleName().equals(helpOption);
-                }
-              }
-          });
-        try {
-          printHelp(printStream, Iterables.getOnlyElement(matches));
-        } catch (NoSuchElementException exception) {
-          printStream.format("Unable to find option %s.%n", helpOption);
-          printHelp(printStream);
-        } catch (IllegalArgumentException exception) {
-          printStream.format("Multiple matches found for %s: %s.%n", helpOption,
-              Iterables.transform(matches, ReflectHelpers.CLASS_NAME));
-          printHelp(printStream);
-        }
-      }
-      if (exit) {
-        System.exit(0);
-      } else {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Returns the simple name of the calling class using the current threads stack.
-   */
-  private static String findCallersClassName() {
-    Iterator<StackTraceElement> elements =
-        Iterators.forArray(Thread.currentThread().getStackTrace());
-    // First find the PipelineOptionsFactory/Builder class in the stack trace.
-    while (elements.hasNext()) {
-      StackTraceElement next = elements.next();
-      if (PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName())) {
-        break;
-      }
-    }
-    // Then find the first instance after that is not the PipelineOptionsFactory/Builder class.
-    while (elements.hasNext()) {
-      StackTraceElement next = elements.next();
-      if (!PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName())) {
-        try {
-          return Class.forName(next.getClassName()).getSimpleName();
-        } catch (ClassNotFoundException e) {
-          break;
-        }
-      }
-    }
-
-    return "unknown";
-  }
-
-  /**
-   * Stores the generated proxyClass and its respective {@link BeanInfo} object.
-   *
-   * @param <T> The type of the proxyClass.
-   */
-  static class Registration<T extends PipelineOptions> {
-    private final Class<T> proxyClass;
-    private final List<PropertyDescriptor> propertyDescriptors;
-
-    public Registration(Class<T> proxyClass, List<PropertyDescriptor> beanInfo) {
-      this.proxyClass = proxyClass;
-      this.propertyDescriptors = beanInfo;
-    }
-
-    List<PropertyDescriptor> getPropertyDescriptors() {
-      return propertyDescriptors;
-    }
-
-    Class<T> getProxyClass() {
-      return proxyClass;
-    }
-  }
-
-  private static final Set<Class<?>> SIMPLE_TYPES = ImmutableSet.<Class<?>>builder()
-      .add(boolean.class)
-      .add(Boolean.class)
-      .add(char.class)
-      .add(Character.class)
-      .add(short.class)
-      .add(Short.class)
-      .add(int.class)
-      .add(Integer.class)
-      .add(long.class)
-      .add(Long.class)
-      .add(float.class)
-      .add(Float.class)
-      .add(double.class)
-      .add(Double.class)
-      .add(String.class)
-      .add(Class.class).build();
-  private static final Logger LOG = LoggerFactory.getLogger(PipelineOptionsFactory.class);
-  @SuppressWarnings("rawtypes")
-  private static final Class<?>[] EMPTY_CLASS_ARRAY = new Class[0];
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-  private static final Map<String, Class<? extends PipelineRunner<?>>> SUPPORTED_PIPELINE_RUNNERS;
-
-  /** Classes that are used as the boundary in the stack trace to find the callers class name. */
-  private static final Set<String> PIPELINE_OPTIONS_FACTORY_CLASSES =
-      ImmutableSet.of(PipelineOptionsFactory.class.getName(), Builder.class.getName());
-
-  /** Methods that are ignored when validating the proxy class. */
-  private static final Set<Method> IGNORED_METHODS;
-
-  /** A predicate that checks if a method is synthetic via {@link Method#isSynthetic()}. */
-  private static final Predicate<Method> NOT_SYNTHETIC_PREDICATE =
-      new Predicate<Method>() {
-        @Override
-        public boolean apply(Method input) {
-          return !input.isSynthetic();
-        }
-      };
-
-  /** The set of options that have been registered and visible to the user. */
-  private static final Set<Class<? extends PipelineOptions>> REGISTERED_OPTIONS =
-      Sets.newConcurrentHashSet();
-
-  /** A cache storing a mapping from a given interface to its registration record. */
-  private static final Map<Class<? extends PipelineOptions>, Registration<?>> INTERFACE_CACHE =
-      Maps.newConcurrentMap();
-
-  /** A cache storing a mapping from a set of interfaces to its registration record. */
-  private static final Map<Set<Class<? extends PipelineOptions>>, Registration<?>> COMBINED_CACHE =
-      Maps.newConcurrentMap();
-
-  /** The width at which options should be output. */
-  private static final int TERMINAL_WIDTH = 80;
-
-  /**
-   * Finds the appropriate {@code ClassLoader} to be used by the
-   * {@link ServiceLoader#load} call, which by default would use the context
-   * {@code ClassLoader}, which can be null. The fallback is as follows: context
-   * ClassLoader, class ClassLoader and finaly the system ClassLoader.
-   */
-  static ClassLoader findClassLoader() {
-    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-    if (classLoader == null) {
-      classLoader = PipelineOptionsFactory.class.getClassLoader();
-    }
-    if (classLoader == null) {
-      classLoader = ClassLoader.getSystemClassLoader();
-    }
-    return classLoader;
-  }
-
-  static {
-    try {
-      IGNORED_METHODS = ImmutableSet.<Method>builder()
-          .add(Object.class.getMethod("getClass"))
-          .add(Object.class.getMethod("wait"))
-          .add(Object.class.getMethod("wait", long.class))
-          .add(Object.class.getMethod("wait", long.class, int.class))
-          .add(Object.class.getMethod("notify"))
-          .add(Object.class.getMethod("notifyAll"))
-          .add(Proxy.class.getMethod("getInvocationHandler", Object.class))
-          .build();
-    } catch (NoSuchMethodException | SecurityException e) {
-      LOG.error("Unable to find expected method", e);
-      throw new ExceptionInInitializerError(e);
-    }
-
-    ClassLoader classLoader = findClassLoader();
-
-    // Store the list of all available pipeline runners.
-    ImmutableMap.Builder<String, Class<? extends PipelineRunner<?>>> builder =
-            ImmutableMap.builder();
-    Set<PipelineRunnerRegistrar> pipelineRunnerRegistrars =
-        Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
-    pipelineRunnerRegistrars.addAll(
-        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class, classLoader)));
-    for (PipelineRunnerRegistrar registrar : pipelineRunnerRegistrars) {
-      for (Class<? extends PipelineRunner<?>> klass : registrar.getPipelineRunners()) {
-        builder.put(klass.getSimpleName(), klass);
-      }
-    }
-    SUPPORTED_PIPELINE_RUNNERS = builder.build();
-
-    // Load and register the list of all classes that extend PipelineOptions.
-    register(PipelineOptions.class);
-    Set<PipelineOptionsRegistrar> pipelineOptionsRegistrars =
-        Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
-    pipelineOptionsRegistrars.addAll(
-        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class, classLoader)));
-    for (PipelineOptionsRegistrar registrar : pipelineOptionsRegistrars) {
-      for (Class<? extends PipelineOptions> klass : registrar.getPipelineOptions()) {
-        register(klass);
-      }
-    }
-  }
-
-  /**
-   * This registers the interface with this factory. This interface must conform to the following
-   * restrictions:
-   * <ul>
-   *   <li>Any property with the same name must have the same return type for all derived
-   *       interfaces of {@link PipelineOptions}.
-   *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
-   *       getter and setter method.
-   *   <li>Every method must conform to being a getter or setter for a JavaBean.
-   *   <li>The derived interface of {@link PipelineOptions} must be composable with every interface
-   *       registered with this factory.
-   * </ul>
-   *
-   * @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.");
-
-    if (REGISTERED_OPTIONS.contains(iface)) {
-      return;
-    }
-    validateWellFormed(iface, REGISTERED_OPTIONS);
-    REGISTERED_OPTIONS.add(iface);
-  }
-
-  /**
-   * Validates that the interface conforms to the following:
-   * <ul>
-   *   <li>Any property with the same name must have the same return type for all derived
-   *       interfaces of {@link PipelineOptions}.
-   *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
-   *       getter and setter method.
-   *   <li>Every method must conform to being a getter or setter for a JavaBean.
-   *   <li>The derived interface of {@link PipelineOptions} must be composable with every interface
-   *       part of allPipelineOptionsClasses.
-   *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
-   *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for
-   *       this property must be annotated with {@link JsonIgnore @JsonIgnore}.
-   * </ul>
-   *
-   * @param iface The interface to validate.
-   * @param validatedPipelineOptionsInterfaces The set of validated pipeline options interfaces to
-   *        validate against.
-   * @return A registration record containing the proxy class and bean info for iface.
-   */
-  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.");
-
-    @SuppressWarnings("unchecked")
-    Set<Class<? extends PipelineOptions>> combinedPipelineOptionsInterfaces =
-        FluentIterable.from(validatedPipelineOptionsInterfaces).append(iface).toSet();
-    // Validate that the view of all currently passed in options classes is well formed.
-    if (!COMBINED_CACHE.containsKey(combinedPipelineOptionsInterfaces)) {
-      @SuppressWarnings("unchecked")
-      Class<T> allProxyClass =
-          (Class<T>) Proxy.getProxyClass(PipelineOptionsFactory.class.getClassLoader(),
-              combinedPipelineOptionsInterfaces.toArray(EMPTY_CLASS_ARRAY));
-      try {
-        List<PropertyDescriptor> propertyDescriptors =
-            validateClass(iface, validatedPipelineOptionsInterfaces, allProxyClass);
-        COMBINED_CACHE.put(combinedPipelineOptionsInterfaces,
-            new Registration<T>(allProxyClass, propertyDescriptors));
-      } catch (IntrospectionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    // Validate that the local view of the class is well formed.
-    if (!INTERFACE_CACHE.containsKey(iface)) {
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      Class<T> proxyClass = (Class<T>) Proxy.getProxyClass(
-          PipelineOptionsFactory.class.getClassLoader(), new Class[] {iface});
-      try {
-        List<PropertyDescriptor> propertyDescriptors =
-            validateClass(iface, validatedPipelineOptionsInterfaces, proxyClass);
-        INTERFACE_CACHE.put(iface,
-            new Registration<T>(proxyClass, propertyDescriptors));
-      } catch (IntrospectionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    @SuppressWarnings("unchecked")
-    Registration<T> result = (Registration<T>) INTERFACE_CACHE.get(iface);
-    return result;
-  }
-
-  public static Set<Class<? extends PipelineOptions>> getRegisteredOptions() {
-    return Collections.unmodifiableSet(REGISTERED_OPTIONS);
-  }
-
-  /**
-   * Outputs the set of registered options with the PipelineOptionsFactory
-   * with a description for each one if available to the output stream. This output
-   * is pretty printed and meant to be human readable. This method will attempt to
-   * format its output to be compatible with a terminal window.
-   */
-  public static void printHelp(PrintStream out) {
-    Preconditions.checkNotNull(out);
-    out.println("The set of registered options are:");
-    Set<Class<? extends PipelineOptions>> sortedOptions =
-        new TreeSet<>(ClassNameComparator.INSTANCE);
-    sortedOptions.addAll(REGISTERED_OPTIONS);
-    for (Class<? extends PipelineOptions> kls : sortedOptions) {
-      out.format("  %s%n", kls.getName());
-    }
-    out.format("%nUse --help=<OptionsName> for detailed help. For example:%n"
-        + "  --help=DataflowPipelineOptions <short names valid for registered options>%n"
-        + "  --help=com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions%n");
-  }
-
-  /**
-   * Outputs the set of options available to be set for the passed in {@link PipelineOptions}
-   * interface. The output is in a human readable format. The format is:
-   * <pre>
-   * OptionGroup:
-   *     ... option group description ...
-   *
-   *  --option1={@code <type>} or list of valid enum choices
-   *     Default: value (if available, see {@link Default})
-   *     ... option description ... (if available, see {@link Description})
-   *     Required groups (if available, see {@link Required})
-   *  --option2={@code <type>} or list of valid enum choices
-   *     Default: value (if available, see {@link Default})
-   *     ... option description ... (if available, see {@link Description})
-   *     Required groups (if available, see {@link Required})
-   * </pre>
-   * 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);
-    validateWellFormed(iface, REGISTERED_OPTIONS);
-
-    Iterable<Method> methods =
-        Iterables.filter(
-            ReflectHelpers.getClosureOfMethodsOnInterface(iface), NOT_SYNTHETIC_PREDICATE);
-    ListMultimap<Class<?>, Method> ifaceToMethods = ArrayListMultimap.create();
-    for (Method method : methods) {
-      // Process only methods that are not marked as hidden.
-      if (method.getAnnotation(Hidden.class) == null) {
-        ifaceToMethods.put(method.getDeclaringClass(), method);
-      }
-    }
-    SortedSet<Class<?>> ifaces = new TreeSet<>(ClassNameComparator.INSTANCE);
-    // Keep interfaces that are not marked as hidden.
-    ifaces.addAll(Collections2.filter(ifaceToMethods.keySet(), new Predicate<Class<?>>() {
-      @Override
-      public boolean apply(Class<?> input) {
-        return input.getAnnotation(Hidden.class) == null;
-      }
-    }));
-    for (Class<?> currentIface : ifaces) {
-      Map<String, Method> propertyNamesToGetters =
-          getPropertyNamesToGetters(ifaceToMethods.get(currentIface));
-
-      // Don't output anything if there are no defined options
-      if (propertyNamesToGetters.isEmpty()) {
-        continue;
-      }
-      SortedSetMultimap<String, String> requiredGroupNameToProperties =
-          getRequiredGroupNamesToProperties(propertyNamesToGetters);
-
-      out.format("%s:%n", currentIface.getName());
-      prettyPrintDescription(out, currentIface.getAnnotation(Description.class));
-
-      out.println();
-
-      List<String> lists = Lists.newArrayList(propertyNamesToGetters.keySet());
-      Collections.sort(lists, String.CASE_INSENSITIVE_ORDER);
-      for (String propertyName : lists) {
-        Method method = propertyNamesToGetters.get(propertyName);
-        String printableType = method.getReturnType().getSimpleName();
-        if (method.getReturnType().isEnum()) {
-          printableType = Joiner.on(" | ").join(method.getReturnType().getEnumConstants());
-        }
-        out.format("  --%s=<%s>%n", propertyName, printableType);
-        Optional<String> defaultValue = getDefaultValueFromAnnotation(method);
-        if (defaultValue.isPresent()) {
-          out.format("    Default: %s%n", defaultValue.get());
-        }
-        prettyPrintDescription(out, method.getAnnotation(Description.class));
-        prettyPrintRequiredGroups(out, method.getAnnotation(Validation.Required.class),
-            requiredGroupNameToProperties);
-      }
-      out.println();
-    }
-  }
-
-  /**
-   * Output the requirement groups that the property is a member of, including all properties that
-   * satisfy the group requirement, breaking up long lines on white space characters and attempting
-   * to honor a line limit of {@code TERMINAL_WIDTH}.
-   */
-  private static void prettyPrintRequiredGroups(PrintStream out, Required annotation,
-      SortedSetMultimap<String, String> requiredGroupNameToProperties) {
-    if (annotation == null || annotation.groups() == null) {
-      return;
-    }
-    for (String group : annotation.groups()) {
-      SortedSet<String> groupMembers = requiredGroupNameToProperties.get(group);
-      String requirement;
-      if (groupMembers.size() == 1) {
-        requirement = Iterables.getOnlyElement(groupMembers) + " is required.";
-      } else {
-        requirement = "At least one of " + groupMembers + " is required";
-      }
-      terminalPrettyPrint(out, requirement.split("\\s+"));
-    }
-  }
-
-  /**
-   * Outputs the value of the description, breaking up long lines on white space characters and
-   * attempting to honor a line limit of {@code TERMINAL_WIDTH}.
-   */
-  private static void prettyPrintDescription(PrintStream out, Description description) {
-    if (description == null || description.value() == null) {
-      return;
-    }
-
-    String[] words = description.value().split("\\s+");
-    terminalPrettyPrint(out, words);
-  }
-
-  private static void terminalPrettyPrint(PrintStream out, String[] words) {
-    final String spacing = "   ";
-
-    if (words.length == 0) {
-      return;
-    }
-
-    out.print(spacing);
-    int lineLength = spacing.length();
-    for (int i = 0; i < words.length; ++i) {
-      out.print(" ");
-      out.print(words[i]);
-      lineLength += 1 + words[i].length();
-
-      // If the next word takes us over the terminal width, then goto the next line.
-      if (i + 1 != words.length && words[i + 1].length() + lineLength + 1 > TERMINAL_WIDTH) {
-        out.println();
-        out.print(spacing);
-        lineLength = spacing.length();
-      }
-    }
-    out.println();
-  }
-
-  /**
-   * Returns a string representation of the {@link Default} value on the passed in method.
-   */
-  private static Optional<String> getDefaultValueFromAnnotation(Method method) {
-    for (Annotation annotation : method.getAnnotations()) {
-      if (annotation instanceof Default.Class) {
-        return Optional.of(((Default.Class) annotation).value().getSimpleName());
-      } else if (annotation instanceof Default.String) {
-        return Optional.of(((Default.String) annotation).value());
-      } else if (annotation instanceof Default.Boolean) {
-        return Optional.of(Boolean.toString(((Default.Boolean) annotation).value()));
-      } else if (annotation instanceof Default.Character) {
-        return Optional.of(Character.toString(((Default.Character) annotation).value()));
-      } else if (annotation instanceof Default.Byte) {
-        return Optional.of(Byte.toString(((Default.Byte) annotation).value()));
-      } else if (annotation instanceof Default.Short) {
-        return Optional.of(Short.toString(((Default.Short) annotation).value()));
-      } else if (annotation instanceof Default.Integer) {
-        return Optional.of(Integer.toString(((Default.Integer) annotation).value()));
-      } else if (annotation instanceof Default.Long) {
-        return Optional.of(Long.toString(((Default.Long) annotation).value()));
-      } else if (annotation instanceof Default.Float) {
-        return Optional.of(Float.toString(((Default.Float) annotation).value()));
-      } else if (annotation instanceof Default.Double) {
-        return Optional.of(Double.toString(((Default.Double) annotation).value()));
-      } else if (annotation instanceof Default.Enum) {
-        return Optional.of(((Default.Enum) annotation).value());
-      } else if (annotation instanceof Default.InstanceFactory) {
-        return Optional.of(((Default.InstanceFactory) annotation).value().getSimpleName());
-      }
-    }
-    return Optional.absent();
-  }
-
-  static Map<String, Class<? extends PipelineRunner<?>>> getRegisteredRunners() {
-    return SUPPORTED_PIPELINE_RUNNERS;
-  }
-
-  static List<PropertyDescriptor> getPropertyDescriptors(
-      Set<Class<? extends PipelineOptions>> interfaces) {
-    return COMBINED_CACHE.get(interfaces).getPropertyDescriptors();
-  }
-
-  /**
-   * This method is meant to emulate the behavior of {@link Introspector#getBeanInfo(Class, int)}
-   * to construct the list of {@link PropertyDescriptor}.
-   *
-   * <p>TODO: Swap back to using Introspector once the proxy class issue with AppEngine is
-   * resolved.
-   */
-  private static List<PropertyDescriptor> getPropertyDescriptors(Class<?> beanClass)
-      throws IntrospectionException {
-    // The sorting is important to make this method stable.
-    SortedSet<Method> methods = Sets.newTreeSet(MethodComparator.INSTANCE);
-    methods.addAll(
-        Collections2.filter(Arrays.asList(beanClass.getMethods()), NOT_SYNTHETIC_PREDICATE));
-    SortedMap<String, Method> propertyNamesToGetters = getPropertyNamesToGetters(methods);
-    List<PropertyDescriptor> descriptors = Lists.newArrayList();
-
-    List<TypeMismatch> mismatches = new ArrayList<>();
-    /*
-     * Add all the getter/setter pairs to the list of descriptors removing the getter once
-     * it has been paired up.
-     */
-    for (Method method : methods) {
-      String methodName = method.getName();
-      if (!methodName.startsWith("set")
-          || method.getParameterTypes().length != 1
-          || method.getReturnType() != void.class) {
-        continue;
-      }
-      String propertyName = Introspector.decapitalize(methodName.substring(3));
-      Method getterMethod = propertyNamesToGetters.remove(propertyName);
-
-      // Validate that the getter and setter property types are the same.
-      if (getterMethod != null) {
-        Class<?> getterPropertyType = getterMethod.getReturnType();
-        Class<?> setterPropertyType = method.getParameterTypes()[0];
-        if (getterPropertyType != setterPropertyType) {
-          TypeMismatch mismatch = new TypeMismatch();
-          mismatch.propertyName = propertyName;
-          mismatch.getterPropertyType = getterPropertyType;
-          mismatch.setterPropertyType = setterPropertyType;
-          mismatches.add(mismatch);
-          continue;
-        }
-      }
-
-      descriptors.add(new PropertyDescriptor(
-          propertyName, getterMethod, method));
-    }
-    throwForTypeMismatches(mismatches);
-
-    // Add the remaining getters with missing setters.
-    for (Map.Entry<String, Method> getterToMethod : propertyNamesToGetters.entrySet()) {
-      descriptors.add(new PropertyDescriptor(
-          getterToMethod.getKey(), getterToMethod.getValue(), null));
-    }
-    return descriptors;
-  }
-
-  private static class TypeMismatch {
-    private String propertyName;
-    private Class<?> getterPropertyType;
-    private Class<?> setterPropertyType;
-  }
-
-  private static void throwForTypeMismatches(List<TypeMismatch> mismatches) {
-    if (mismatches.size() == 1) {
-      TypeMismatch mismatch = mismatches.get(0);
-      throw new IllegalArgumentException(String.format(
-          "Type mismatch between getter and setter methods for property [%s]. "
-          + "Getter is of type [%s] whereas setter is of type [%s].",
-          mismatch.propertyName,
-          mismatch.getterPropertyType.getName(),
-          mismatch.setterPropertyType.getName()));
-    } else if (mismatches.size() > 1) {
-      StringBuilder builder = new StringBuilder(
-          String.format("Type mismatches between getters and setters detected:"));
-      for (TypeMismatch mismatch : mismatches) {
-        builder.append(String.format(
-            "%n  - Property [%s]: Getter is of type [%s] whereas setter is of type [%s].",
-            mismatch.propertyName,
-            mismatch.getterPropertyType.getName(),
-            mismatch.setterPropertyType.getName()));
-      }
-      throw new IllegalArgumentException(builder.toString());
-    }
-  }
-
-  /**
-   * Returns a map of the property name to the getter method it represents.
-   * If there are duplicate methods with the same bean name, then it is indeterminate
-   * as to which method will be returned.
-   */
-  private static SortedMap<String, Method> getPropertyNamesToGetters(Iterable<Method> methods) {
-    SortedMap<String, Method> propertyNamesToGetters = Maps.newTreeMap();
-    for (Method method : methods) {
-      String methodName = method.getName();
-      if ((!methodName.startsWith("get")
-          && !methodName.startsWith("is"))
-          || method.getParameterTypes().length != 0
-          || method.getReturnType() == void.class) {
-        continue;
-      }
-      String propertyName = Introspector.decapitalize(
-          methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3));
-      propertyNamesToGetters.put(propertyName, method);
-    }
-    return propertyNamesToGetters;
-  }
-
-  /**
-   * Returns a map of required groups of arguments to the properties that satisfy the requirement.
-   */
-  private static SortedSetMultimap<String, String> getRequiredGroupNamesToProperties(
-      Map<String, Method> propertyNamesToGetters) {
-    SortedSetMultimap<String, String> result = TreeMultimap.create();
-    for (Map.Entry<String, Method> propertyEntry : propertyNamesToGetters.entrySet()) {
-      Required requiredAnnotation =
-          propertyEntry.getValue().getAnnotation(Validation.Required.class);
-      if (requiredAnnotation != null) {
-        for (String groupName : requiredAnnotation.groups()) {
-          result.put(groupName, propertyEntry.getKey());
-        }
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Validates that a given class conforms to the following properties:
-   * <ul>
-   *   <li>Any property with the same name must have the same return type for all derived
-   *       interfaces of {@link PipelineOptions}.
-   *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
-   *       getter and setter method.
-   *   <li>Every method must conform to being a getter or setter for a JavaBean.
-   *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
-   *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for
-   *       this property must be annotated with {@link JsonIgnore @JsonIgnore}.
-   * </ul>
-   *
-   * @param iface The interface to validate.
-   * @param validatedPipelineOptionsInterfaces The set of validated pipeline options interfaces to
-   *        validate against.
-   * @param klass The proxy class representing the interface.
-   * @return A list of {@link PropertyDescriptor}s representing all valid bean properties of
-   *         {@code iface}.
-   * @throws IntrospectionException if invalid property descriptors.
-   */
-  private static List<PropertyDescriptor> validateClass(Class<? extends PipelineOptions> iface,
-      Set<Class<? extends PipelineOptions>> validatedPipelineOptionsInterfaces,
-      Class<?> klass) throws IntrospectionException {
-    Set<Method> methods = Sets.newHashSet(IGNORED_METHODS);
-    // Ignore static methods, "equals", "hashCode", "toString" and "as" on the generated class.
-    // Ignore synthetic methods
-    for (Method method : klass.getMethods()) {
-      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
-        methods.add(method);
-      }
-    }
-    try {
-      methods.add(klass.getMethod("equals", Object.class));
-      methods.add(klass.getMethod("hashCode"));
-      methods.add(klass.getMethod("toString"));
-      methods.add(klass.getMethod("as", Class.class));
-      methods.add(klass.getMethod("cloneAs", Class.class));
-    } catch (NoSuchMethodException | SecurityException e) {
-      throw Throwables.propagate(e);
-    }
-
-    // Verify that there are no methods with the same name with two different return types.
-    Iterable<Method> interfaceMethods = FluentIterable
-        .from(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
-        .filter(NOT_SYNTHETIC_PREDICATE)
-        .toSortedSet(MethodComparator.INSTANCE);
-    SortedSetMultimap<Method, Method> methodNameToMethodMap =
-        TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
-    for (Method method : interfaceMethods) {
-      methodNameToMethodMap.put(method, method);
-    }
-    List<MultipleDefinitions> multipleDefinitions = Lists.newArrayList();
-    for (Map.Entry<Method, Collection<Method>> entry
-        : methodNameToMethodMap.asMap().entrySet()) {
-      Set<Class<?>> returnTypes = FluentIterable.from(entry.getValue())
-          .transform(ReturnTypeFetchingFunction.INSTANCE).toSet();
-      SortedSet<Method> collidingMethods = FluentIterable.from(entry.getValue())
-          .toSortedSet(MethodComparator.INSTANCE);
-      if (returnTypes.size() > 1) {
-        MultipleDefinitions defs = new MultipleDefinitions();
-        defs.method = entry.getKey();
-        defs.collidingMethods = collidingMethods;
-        multipleDefinitions.add(defs);
-      }
-    }
-    throwForMultipleDefinitions(iface, multipleDefinitions);
-
-    // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
-    // that no setter has @JsonIgnore.
-    Iterable<Method> allInterfaceMethods =
-        FluentIterable.from(
-                ReflectHelpers.getClosureOfMethodsOnInterfaces(
-                    validatedPipelineOptionsInterfaces))
-            .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
-            .filter(NOT_SYNTHETIC_PREDICATE)
-            .toSortedSet(MethodComparator.INSTANCE);
-    SortedSetMultimap<Method, Method> methodNameToAllMethodMap =
-        TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
-    for (Method method : allInterfaceMethods) {
-      methodNameToAllMethodMap.put(method, method);
-    }
-
-    List<PropertyDescriptor> descriptors = getPropertyDescriptors(klass);
-
-    List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
-    List<IgnoredSetter> ignoredSetters = new ArrayList<>();
-
-    for (PropertyDescriptor descriptor : descriptors) {
-      if (descriptor.getReadMethod() == null
-          || descriptor.getWriteMethod() == null
-          || IGNORED_METHODS.contains(descriptor.getReadMethod())
-          || IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
-        continue;
-      }
-      SortedSet<Method> getters = methodNameToAllMethodMap.get(descriptor.getReadMethod());
-      SortedSet<Method> gettersWithJsonIgnore = Sets.filter(getters, JsonIgnorePredicate.INSTANCE);
-
-      Iterable<String> getterClassNames = FluentIterable.from(getters)
-          .transform(MethodToDeclaringClassFunction.INSTANCE)
-          .transform(ReflectHelpers.CLASS_NAME);
-      Iterable<String> gettersWithJsonIgnoreClassNames = FluentIterable.from(gettersWithJsonIgnore)
-          .transform(MethodToDeclaringClassFunction.INSTANCE)
-          .transform(ReflectHelpers.CLASS_NAME);
-
-      if (!(gettersWithJsonIgnore.isEmpty() || getters.size() == gettersWithJsonIgnore.size())) {
-        InconsistentlyIgnoredGetters err = new InconsistentlyIgnoredGetters();
-        err.descriptor = descriptor;
-        err.getterClassNames = getterClassNames;
-        err.gettersWithJsonIgnoreClassNames = gettersWithJsonIgnoreClassNames;
-        incompletelyIgnoredGetters.add(err);
-      }
-      if (!incompletelyIgnoredGetters.isEmpty()) {
-        continue;
-      }
-
-      SortedSet<Method> settersWithJsonIgnore =
-          Sets.filter(methodNameToAllMethodMap.get(descriptor.getWriteMethod()),
-              JsonIgnorePredicate.INSTANCE);
-
-      Iterable<String> settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore)
-              .transform(MethodToDeclaringClassFunction.INSTANCE)
-              .transform(ReflectHelpers.CLASS_NAME);
-
-      if (!settersWithJsonIgnore.isEmpty()) {
-        IgnoredSetter ignored = new IgnoredSetter();
-        ignored.descriptor = descriptor;
-        ignored.settersWithJsonIgnoreClassNames = settersWithJsonIgnoreClassNames;
-        ignoredSetters.add(ignored);
-      }
-    }
-    throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters);
-    throwForSettersWithJsonIgnore(ignoredSetters);
-
-    List<MissingBeanMethod> missingBeanMethods = new ArrayList<>();
-    // Verify that each property has a matching read and write method.
-    for (PropertyDescriptor propertyDescriptor : descriptors) {
-      if (!(IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod())
-        || propertyDescriptor.getReadMethod() != null)) {
-        MissingBeanMethod method = new MissingBeanMethod();
-        method.property = propertyDescriptor;
-        method.methodType = "getter";
-        missingBeanMethods.add(method);
-        continue;
-      }
-      if (!(IGNORED_METHODS.contains(propertyDescriptor.getReadMethod())
-              || propertyDescriptor.getWriteMethod() != null)) {
-        MissingBeanMethod method = new MissingBeanMethod();
-        method.property = propertyDescriptor;
-        method.methodType = "setter";
-        missingBeanMethods.add(method);
-        continue;
-      }
-      methods.add(propertyDescriptor.getReadMethod());
-      methods.add(propertyDescriptor.getWriteMethod());
-    }
-    throwForMissingBeanMethod(iface, missingBeanMethods);
-
-    // Verify that no additional methods are on an interface that aren't a bean property.
-    SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
-    unknownMethods.addAll(
-        Sets.filter(
-            Sets.difference(Sets.newHashSet(klass.getMethods()), methods),
-            NOT_SYNTHETIC_PREDICATE));
-    Preconditions.checkArgument(unknownMethods.isEmpty(),
-        "Methods %s on [%s] do not conform to being bean properties.",
-        FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
-        iface.getName());
-
-    return descriptors;
-  }
-
-  private static class MultipleDefinitions {
-    private Method method;
-    private SortedSet<Method> collidingMethods;
-  }
-
-  private static void throwForMultipleDefinitions(
-      Class<? extends PipelineOptions> iface, List<MultipleDefinitions> definitions) {
-    if (definitions.size() == 1) {
-      MultipleDefinitions errDef = definitions.get(0);
-      throw new IllegalArgumentException(String.format(
-          "Method [%s] has multiple definitions %s with different return types for [%s].",
-          errDef.method.getName(), errDef.collidingMethods, iface.getName()));
-    } else if (definitions.size() > 1) {
-      StringBuilder errorBuilder = new StringBuilder(String.format(
-          "Interface [%s] has Methods with multiple definitions with different return types:",
-          iface.getName()));
-      for (MultipleDefinitions errDef : definitions) {
-        errorBuilder.append(String.format(
-            "%n  - Method [%s] has multiple definitions %s",
-            errDef.method.getName(),
-            errDef.collidingMethods));
-      }
-      throw new IllegalArgumentException(errorBuilder.toString());
-    }
-  }
-
-  private static class InconsistentlyIgnoredGetters {
-    PropertyDescriptor descriptor;
-    Iterable<String> getterClassNames;
-    Iterable<String> gettersWithJsonIgnoreClassNames;
-  }
-
-  private static void throwForGettersWithInconsistentJsonIgnore(
-      List<InconsistentlyIgnoredGetters> getters) {
-    if (getters.size() == 1) {
-      InconsistentlyIgnoredGetters getter = getters.get(0);
-      throw new IllegalArgumentException(String.format(
-          "Expected getter for property [%s] to be marked with @JsonIgnore on all %s, "
-          + "found only on %s",
-          getter.descriptor.getName(), getter.getterClassNames,
-          getter.gettersWithJsonIgnoreClassNames));
-    } else if (getters.size() > 1) {
-      StringBuilder errorBuilder =
-          new StringBuilder("Property getters are inconsistently marked with @JsonIgnore:");
-      for (InconsistentlyIgnoredGetters getter : getters) {
-        errorBuilder.append(
-            String.format("%n  - Expected for property [%s] to be marked on all %s, "
-                + "found only on %s",
-                getter.descriptor.getName(), getter.getterClassNames,
-                getter.gettersWithJsonIgnoreClassNames));
-      }
-      throw new IllegalArgumentException(errorBuilder.toString());
-    }
-  }
-
-  private static class IgnoredSetter {
-    PropertyDescriptor descriptor;
-    Iterable<String> settersWithJsonIgnoreClassNames;
-  }
-
-  private static void throwForSettersWithJsonIgnore(List<IgnoredSetter> setters) {
-    if (setters.size() == 1) {
-      IgnoredSetter setter = setters.get(0);
-      throw new IllegalArgumentException(
-          String.format("Expected setter for property [%s] to not be marked with @JsonIgnore on %s",
-              setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
-    } else if (setters.size() > 1) {
-      StringBuilder builder = new StringBuilder("Found setters marked with @JsonIgnore:");
-      for (IgnoredSetter setter : setters) {
-        builder.append(
-            String.format("%n  - Setter for property [%s] should not be marked with @JsonIgnore "
-                + "on %s",
-                setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
-      }
-      throw new IllegalArgumentException(builder.toString());
-    }
-  }
-
-  private static class MissingBeanMethod {
-    String methodType;
-    PropertyDescriptor property;
-  }
-
-  private static void throwForMissingBeanMethod(
-      Class<? extends PipelineOptions> iface, List<MissingBeanMethod> missingBeanMethods) {
-    if (missingBeanMethods.size() == 1) {
-      MissingBeanMethod missingBeanMethod = missingBeanMethods.get(0);
-      throw new IllegalArgumentException(
-          String.format("Expected %s for property [%s] of type [%s] on [%s].",
-              missingBeanMethod.methodType, missingBeanMethod.property.getName(),
-              missingBeanMethod.property.getPropertyType().getName(), iface.getName()));
-    } else if (missingBeanMethods.size() > 1) {
-      StringBuilder builder = new StringBuilder(String.format(
-          "Found missing property methods on [%s]:", iface.getName()));
-      for (MissingBeanMethod method : missingBeanMethods) {
-        builder.append(
-            String.format("%n  - Expected %s for property [%s] of type [%s]", method.methodType,
-                method.property.getName(), method.property.getPropertyType().getName()));
-      }
-      throw new IllegalArgumentException(builder.toString());
-    }
-  }
-
-  /** A {@link Comparator} that uses the classes name to compare them. */
-  private static class ClassNameComparator implements Comparator<Class<?>> {
-    static final ClassNameComparator INSTANCE = new ClassNameComparator();
-    @Override
-    public int compare(Class<?> o1, Class<?> o2) {
-      return o1.getName().compareTo(o2.getName());
-    }
-  }
-
-  /** A {@link Comparator} that uses the object's classes canonical name to compare them. */
-  private static class ObjectsClassComparator implements Comparator<Object> {
-    static final ObjectsClassComparator INSTANCE = new ObjectsClassComparator();
-    @Override
-    public int compare(Object o1, Object o2) {
-      return o1.getClass().getCanonicalName().compareTo(o2.getClass().getCanonicalName());
-    }
-  }
-
-  /** A {@link Comparator} that uses the generic method signature to sort them. */
-  private static class MethodComparator implements Comparator<Method> {
-    static final MethodComparator INSTANCE = new MethodComparator();
-    @Override
-    public int compare(Method o1, Method o2) {
-      return o1.toGenericString().compareTo(o2.toGenericString());
-    }
-  }
-
-  /** A {@link Comparator} that uses the methods name to compare them. */
-  static class MethodNameComparator implements Comparator<Method> {
-    static final MethodNameComparator INSTANCE = new MethodNameComparator();
-    @Override
-    public int compare(Method o1, Method o2) {
-      return o1.getName().compareTo(o2.getName());
-    }
-  }
-
-  /** A {@link Function} that gets the method's return type. */
-  private static class ReturnTypeFetchingFunction implements Function<Method, Class<?>> {
-    static final ReturnTypeFetchingFunction INSTANCE = new ReturnTypeFetchingFunction();
-    @Override
-    public Class<?> apply(Method input) {
-      return input.getReturnType();
-    }
-  }
-
-  /** A {@link Function} with returns the declaring class for the method. */
-  private static class MethodToDeclaringClassFunction implements Function<Method, Class<?>> {
-    static final MethodToDeclaringClassFunction INSTANCE = new MethodToDeclaringClassFunction();
-    @Override
-    public Class<?> apply(Method input) {
-      return input.getDeclaringClass();
-    }
-  }
-
-  /**
-   * A {@link Predicate} that returns true if the method is annotated with
-   * {@link JsonIgnore @JsonIgnore}.
-   */
-  static class JsonIgnorePredicate implements Predicate<Method> {
-    static final JsonIgnorePredicate INSTANCE = new JsonIgnorePredicate();
-    @Override
-    public boolean apply(Method input) {
-      return input.isAnnotationPresent(JsonIgnore.class);
-    }
-  }
-
-  /**
-   * Splits string arguments based upon expected pattern of --argName=value.
-   *
-   * <p>Example GNU style command line arguments:
-   *
-   * <pre>
-   *   --project=MyProject (simple property, will set the "project" property to "MyProject")
-   *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
-   *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
-   *   --x=1 --x=2 --x=3 (list style simple property, will set the "x" property to [1, 2, 3])
-   *   --x=1,2,3 (shorthand list style simple property, will set the "x" property to [1, 2, 3])
-   *   --complexObject='{"key1":"value1",...} (JSON format for all other complex types)
-   * </pre>
-   *
-   * <p>Simple properties are able to bound to {@link String}, {@link Class}, enums and Java
-   * primitives {@code boolean}, {@code byte}, {@code short}, {@code int}, {@code long},
-   * {@code float}, {@code double} and their primitive wrapper classes.
-   *
-   * <p>Simple list style properties are able to be bound to {@code boolean[]}, {@code char[]},
-   * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]},
-   * {@code Class[]}, enum arrays, {@code String[]}, and {@code List<String>}.
-   *
-   * <p>JSON format is required for all other types.
-   *
-   * <p>If strict parsing is enabled, options must start with '--', and not have an empty argument
-   * name or value based upon the positioning of the '='. Empty or null arguments will be ignored
-   * whether or not strict parsing is enabled.
-   */
-  private static ListMultimap<String, String> parseCommandLine(
-      String[] args, boolean strictParsing) {
-    ImmutableListMultimap.Builder<String, String> builder = ImmutableListMultimap.builder();
-    for (String arg : args) {
-      if (Strings.isNullOrEmpty(arg)) {
-        continue;
-      }
-      try {
-        Preconditions.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,
-            "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()));
-        } else {
-          builder.put(arg.substring(2), "true");
-        }
-      } catch (IllegalArgumentException e) {
-        if (strictParsing) {
-          throw e;
-        } else {
-          LOG.warn("Strict parsing is disabled, ignoring option '{}' because {}",
-              arg, e.getMessage());
-        }
-      }
-    }
-    return builder.build();
-  }
-
-  /**
-   * Using the parsed string arguments, we convert the strings to the expected
-   * return type of the methods that are found on the passed-in class.
-   *
-   * <p>For any return type that is expected to be an array or a collection, we further
-   * split up each string on ','.
-   *
-   * <p>We special case the "runner" option. It is mapped to the class of the {@link PipelineRunner}
-   * based off of the {@link PipelineRunner PipelineRunners} simple class name. If the provided
-   * runner name is not registered via a {@link PipelineRunnerRegistrar}, we attempt to obtain the
-   * class that the name represents using {@link Class#forName(String)} and use the result class if
-   * it subclasses {@link PipelineRunner}.
-   *
-   * <p>If strict parsing is enabled, unknown options or options that cannot be converted to
-   * the expected java type using an {@link ObjectMapper} will be ignored.
-   */
-  private static <T extends PipelineOptions> Map<String, Object> parseObjects(
-      Class<T> klass, ListMultimap<String, String> options, boolean strictParsing) {
-    Map<String, Method> propertyNamesToGetters = Maps.newHashMap();
-    PipelineOptionsFactory.validateWellFormed(klass, REGISTERED_OPTIONS);
-    @SuppressWarnings("unchecked")
-    Iterable<PropertyDescriptor> propertyDescriptors =
-        PipelineOptionsFactory.getPropertyDescriptors(
-            FluentIterable.from(getRegisteredOptions()).append(klass).toSet());
-    for (PropertyDescriptor descriptor : propertyDescriptors) {
-      propertyNamesToGetters.put(descriptor.getName(), descriptor.getReadMethod());
-    }
-    Map<String, Object> convertedOptions = Maps.newHashMap();
-    for (final Map.Entry<String, Collection<String>> entry : options.asMap().entrySet()) {
-      try {
-        // Search for close matches for missing properties.
-        // Either off by one or off by two character errors.
-        if (!propertyNamesToGetters.containsKey(entry.getKey())) {
-          SortedSet<String> closestMatches = new TreeSet<String>(
-              Sets.filter(propertyNamesToGetters.keySet(), new Predicate<String>() {
-                @Override
-                public boolean apply(@Nullable String input) {
-                  return StringUtils.getLevenshteinDistance(entry.getKey(), input) <= 2;
-                }
-          }));
-          switch (closestMatches.size()) {
-            case 0:
-              throw new IllegalArgumentException(
-                  String.format("Class %s missing a property named '%s'.",
-                      klass, entry.getKey()));
-            case 1:
-              throw new IllegalArgumentException(
-                  String.format("Class %s missing a property named '%s'. Did you mean '%s'?",
-                      klass, entry.getKey(), Iterables.getOnlyElement(closestMatches)));
-            default:
-              throw new IllegalArgumentException(
-                  String.format("Class %s missing a property named '%s'. Did you mean one of %s?",
-                      klass, entry.getKey(), closestMatches));
-          }
-        }
-
-        Method method = propertyNamesToGetters.get(entry.getKey());
-        // Only allow empty argument values for String, String Array, and Collection.
-        Class<?> returnType = method.getReturnType();
-        JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType());
-        if ("runner".equals(entry.getKey())) {
-          String runner = Iterables.getOnlyElement(entry.getValue());
-          if (SUPPORTED_PIPELINE_RUNNERS.containsKey(runner)) {
-            convertedOptions.put("runner", SUPPORTED_PIPELINE_RUNNERS.get(runner));
-          } else {
-            try {
-              Class<?> runnerClass = Class.forName(runner);
-              checkArgument(
-                  PipelineRunner.class.isAssignableFrom(runnerClass),
-                  "Class '%s' does not implement PipelineRunner. Supported pipeline runners %s",
-                  runner,
-                  Sets.newTreeSet(SUPPORTED_PIPELINE_RUNNERS.keySet()));
-              convertedOptions.put("runner", runnerClass);
-            } catch (ClassNotFoundException e) {
-              String msg =
-                  String.format(
-                      "Unknown 'runner' specified '%s', supported pipeline runners %s",
-                      runner,
-                      Sets.newTreeSet(SUPPORTED_PIPELINE_RUNNERS.keySet()));
-                throw new IllegalArgumentException(msg, e);
-            }
-          }
-        } else if ((returnType.isArray() && (SIMPLE_TYPES.contains(returnType.getComponentType())
-                || returnType.getComponentType().isEnum()))
-            || Collection.class.isAssignableFrom(returnType)) {
-          // Split any strings with ","
-          List<String> values = FluentIterable.from(entry.getValue())
-              .transformAndConcat(new Function<String, Iterable<String>>() {
-                @Override
-                public Iterable<String> apply(String input) {
-                  return Arrays.asList(input.split(","));
-                }
-          }).toList();
-
-          if (returnType.isArray() && !returnType.getComponentType().equals(String.class)) {
-            for (String value : values) {
-              Preconditions.checkArgument(!value.isEmpty(),
-                  "Empty argument value is only allowed for String, String Array, and Collection,"
-                  + " but received: " + returnType);
-            }
-          }
-          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(),
-              "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(),
-              "Empty argument value is only allowed for String, String Array, and Collection,"
-               + " but received: " + returnType);
-          try {
-            convertedOptions.put(entry.getKey(), MAPPER.readValue(value, type));
-          } catch (IOException e) {
-            throw new IllegalArgumentException("Unable to parse JSON value " + value, e);
-          }
-        }
-      } catch (IllegalArgumentException e) {
-        if (strictParsing) {
-          throw e;
-        } else {
-          LOG.warn("Strict parsing is disabled, ignoring option '{}' with value '{}' because {}",
-              entry.getKey(), entry.getValue(), e.getMessage());
-        }
-      }
-    }
-    return convertedOptions;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java
deleted file mode 100644
index 3fbe7ca..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java
+++ /dev/null
@@ -1,37 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.auto.service.AutoService;
-
-import java.util.ServiceLoader;
-
-/**
- * {@link PipelineOptions} creators have the ability to automatically have their
- * {@link PipelineOptions} registered with this SDK by creating a {@link ServiceLoader} entry
- * and a concrete implementation of this interface.
- *
- * <p>Note that automatic registration of any {@link PipelineOptions} requires users
- * conform to the limitations discussed on {@link PipelineOptionsFactory#register(Class)}.
- *
- * <p>It is optional but recommended to use one of the many build time tools such as
- * {@link AutoService} to generate the necessary META-INF files automatically.
- */
-public interface PipelineOptionsRegistrar {
-  Iterable<Class<? extends PipelineOptions>> getPipelineOptions();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
deleted file mode 100644
index 9143c0b..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
+++ /dev/null
@@ -1,103 +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 com.google.cloud.dataflow.sdk.options;
-
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.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;
-import com.google.common.collect.TreeMultimap;
-
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.util.Collection;
-
-/**
- * Validates that the {@link PipelineOptions} conforms to all the {@link Validation} criteria.
- */
-public class PipelineOptionsValidator {
-  /**
-   * Validates that the passed {@link PipelineOptions} conforms to all the validation criteria from
-   * the passed in interface.
-   *
-   * <p>Note that the interface requested must conform to the validation criteria specified on
-   * {@link PipelineOptions#as(Class)}.
-   *
-   * @param klass The interface to fetch validation criteria from.
-   * @param options The {@link PipelineOptions} to validate.
-   * @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);
-
-    // Ensure the methods for T are registered on the ProxyInvocationHandler
-    T asClassOptions = options.as(klass);
-
-    ProxyInvocationHandler handler =
-        (ProxyInvocationHandler) Proxy.getInvocationHandler(asClassOptions);
-
-    SortedSetMultimap<String, Method> requiredGroups = TreeMultimap.create(
-        Ordering.natural(), PipelineOptionsFactory.MethodNameComparator.INSTANCE);
-    for (Method method : ReflectHelpers.getClosureOfMethodsOnInterface(klass)) {
-      Required requiredAnnotation = method.getAnnotation(Validation.Required.class);
-      if (requiredAnnotation != null) {
-        if (requiredAnnotation.groups().length > 0) {
-          for (String requiredGroup : requiredAnnotation.groups()) {
-            requiredGroups.put(requiredGroup, method);
-          }
-        } else {
-          Preconditions.checkArgument(handler.invoke(asClassOptions, method, null) != null,
-              "Missing required value for [" + method + ", \"" + getDescription(method) + "\"]. ");
-        }
-      }
-    }
-
-    for (String requiredGroup : requiredGroups.keySet()) {
-      if (!verifyGroup(handler, asClassOptions, requiredGroups.get(requiredGroup))) {
-        throw new IllegalArgumentException("Missing required value for group [" + requiredGroup
-            + "]. At least one of the following properties "
-            + Collections2.transform(
-                requiredGroups.get(requiredGroup), ReflectHelpers.METHOD_FORMATTER)
-            + " required. Run with --help=" + klass.getSimpleName() + " for more information.");
-      }
-    }
-
-    return asClassOptions;
-  }
-
-  private static boolean verifyGroup(ProxyInvocationHandler handler, PipelineOptions options,
-      Collection<Method> requiredGroup) {
-    for (Method m : requiredGroup) {
-      if (handler.invoke(options, m, null) != null) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private static String getDescription(Method method) {
-    Description description = method.getAnnotation(Description.class);
-    return description == null ? "" : description.value();
-  }
-}



[42/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java
deleted file mode 100644
index b1f4f27..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java
+++ /dev/null
@@ -1,86 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.examples.WordCount.CountWords;
-import com.google.cloud.dataflow.examples.WordCount.ExtractWordsFn;
-import com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Tests of WordCount.
- */
-@RunWith(JUnit4.class)
-public class WordCountTest {
-
-  /** Example test that tests a specific DoFn. */
-  @Test
-  public void testExtractWordsFn() {
-    DoFnTester<String, String> extractWordsFn =
-        DoFnTester.of(new ExtractWordsFn());
-
-    Assert.assertThat(extractWordsFn.processBatch(" some  input  words "),
-                      CoreMatchers.hasItems("some", "input", "words"));
-    Assert.assertThat(extractWordsFn.processBatch(" "),
-                      CoreMatchers.<String>hasItems());
-    Assert.assertThat(extractWordsFn.processBatch(" some ", " input", " words"),
-                      CoreMatchers.hasItems("some", "input", "words"));
-  }
-
-  static final String[] WORDS_ARRAY = new String[] {
-    "hi there", "hi", "hi sue bob",
-    "hi sue", "", "bob hi"};
-
-  static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
-  static final String[] COUNTS_ARRAY = new String[] {
-      "hi: 5", "there: 1", "sue: 2", "bob: 2"};
-
-  /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testCountWords() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()));
-
-    PCollection<String> output = input.apply(new CountWords())
-      .apply(MapElements.via(new FormatAsTextFn()));
-
-    PAssert.that(output).containsInAnyOrder(COUNTS_ARRAY);
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java
deleted file mode 100644
index a70aee1..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java
+++ /dev/null
@@ -1,182 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.cloud.dataflow.examples.complete.AutoComplete.CompletionCandidate;
-import com.google.cloud.dataflow.examples.complete.AutoComplete.ComputeTopCompletions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * Tests of AutoComplete.
- */
-@RunWith(Parameterized.class)
-public class AutoCompleteTest implements Serializable {
-  private boolean recursive;
-
-  public AutoCompleteTest(Boolean recursive) {
-    this.recursive = recursive;
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> testRecursive() {
-    return Arrays.asList(new Object[][] {
-        { true },
-        { false }
-      });
-  }
-
-  @Test
-  public void testAutoComplete() {
-    List<String> words = Arrays.asList(
-        "apple",
-        "apple",
-        "apricot",
-        "banana",
-        "blackberry",
-        "blackberry",
-        "blackberry",
-        "blueberry",
-        "blueberry",
-        "cherry");
-
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(words));
-
-    PCollection<KV<String, List<CompletionCandidate>>> output =
-      input.apply(new ComputeTopCompletions(2, recursive))
-           .apply(Filter.byPredicate(
-                        new SerializableFunction<KV<String, List<CompletionCandidate>>, Boolean>() {
-                          @Override
-                          public Boolean apply(KV<String, List<CompletionCandidate>> element) {
-                            return element.getKey().length() <= 2;
-                          }
-                      }));
-
-    PAssert.that(output).containsInAnyOrder(
-        KV.of("a", parseList("apple:2", "apricot:1")),
-        KV.of("ap", parseList("apple:2", "apricot:1")),
-        KV.of("b", parseList("blackberry:3", "blueberry:2")),
-        KV.of("ba", parseList("banana:1")),
-        KV.of("bl", parseList("blackberry:3", "blueberry:2")),
-        KV.of("c", parseList("cherry:1")),
-        KV.of("ch", parseList("cherry:1")));
-    p.run();
-  }
-
-  @Test
-  public void testTinyAutoComplete() {
-    List<String> words = Arrays.asList("x", "x", "x", "xy", "xy", "xyz");
-
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(words));
-
-    PCollection<KV<String, List<CompletionCandidate>>> output =
-      input.apply(new ComputeTopCompletions(2, recursive));
-
-    PAssert.that(output).containsInAnyOrder(
-        KV.of("x", parseList("x:3", "xy:2")),
-        KV.of("xy", parseList("xy:2", "xyz:1")),
-        KV.of("xyz", parseList("xyz:1")));
-    p.run();
-  }
-
-  @Test
-  public void testWindowedAutoComplete() {
-    List<TimestampedValue<String>> words = Arrays.asList(
-        TimestampedValue.of("xA", new Instant(1)),
-        TimestampedValue.of("xA", new Instant(1)),
-        TimestampedValue.of("xB", new Instant(1)),
-        TimestampedValue.of("xB", new Instant(2)),
-        TimestampedValue.of("xB", new Instant(2)));
-
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p
-      .apply(Create.of(words))
-      .apply(new ReifyTimestamps<String>());
-
-    PCollection<KV<String, List<CompletionCandidate>>> output =
-      input.apply(Window.<String>into(SlidingWindows.of(new Duration(2))))
-           .apply(new ComputeTopCompletions(2, recursive));
-
-    PAssert.that(output).containsInAnyOrder(
-        // Window [0, 2)
-        KV.of("x", parseList("xA:2", "xB:1")),
-        KV.of("xA", parseList("xA:2")),
-        KV.of("xB", parseList("xB:1")),
-
-        // Window [1, 3)
-        KV.of("x", parseList("xB:3", "xA:2")),
-        KV.of("xA", parseList("xA:2")),
-        KV.of("xB", parseList("xB:3")),
-
-        // Window [2, 3)
-        KV.of("x", parseList("xB:2")),
-        KV.of("xB", parseList("xB:2")));
-    p.run();
-  }
-
-  private static List<CompletionCandidate> parseList(String... entries) {
-    List<CompletionCandidate> all = new ArrayList<>();
-    for (String s : entries) {
-      String[] countValue = s.split(":");
-      all.add(new CompletionCandidate(countValue[0], Integer.valueOf(countValue[1])));
-    }
-    return all;
-  }
-
-  private static class ReifyTimestamps<T>
-      extends PTransform<PCollection<TimestampedValue<T>>, PCollection<T>> {
-    @Override
-    public PCollection<T> apply(PCollection<TimestampedValue<T>> input) {
-      return input.apply(ParDo.of(new DoFn<TimestampedValue<T>, T>() {
-        @Override
-        public void processElement(ProcessContext c) {
-          c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp());
-        }
-      }));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TfIdfTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TfIdfTest.java
deleted file mode 100644
index 5989ce8..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TfIdfTest.java
+++ /dev/null
@@ -1,68 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.net.URI;
-import java.util.Arrays;
-
-/**
- * Tests of {@link TfIdf}.
- */
-@RunWith(JUnit4.class)
-public class TfIdfTest {
-
-  /** Test that the example runs. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testTfIdf() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
-    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
-    PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
-        .apply(Create.of(
-            KV.of(new URI("x"), "a b c d"),
-            KV.of(new URI("y"), "a b c"),
-            KV.of(new URI("z"), "a m n")))
-        .apply(new TfIdf.ComputeTfIdf());
-
-    PCollection<String> words = wordToUriAndTfIdf
-        .apply(Keys.<String>create())
-        .apply(RemoveDuplicates.<String>create());
-
-    PAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d"));
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessionsTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessionsTest.java
deleted file mode 100644
index 52f69b0..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessionsTest.java
+++ /dev/null
@@ -1,63 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-
-/** Unit tests for {@link TopWikipediaSessions}. */
-@RunWith(JUnit4.class)
-public class TopWikipediaSessionsTest {
-  @Test
-  @Category(RunnableOnService.class)
-  public void testComputeTopUsers() {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> output =
-        p.apply(Create.of(Arrays.asList(
-            new TableRow().set("timestamp", 0).set("contributor_username", "user1"),
-            new TableRow().set("timestamp", 1).set("contributor_username", "user1"),
-            new TableRow().set("timestamp", 2).set("contributor_username", "user1"),
-            new TableRow().set("timestamp", 0).set("contributor_username", "user2"),
-            new TableRow().set("timestamp", 1).set("contributor_username", "user2"),
-            new TableRow().set("timestamp", 3601).set("contributor_username", "user2"),
-            new TableRow().set("timestamp", 3602).set("contributor_username", "user2"),
-            new TableRow().set("timestamp", 35 * 24 * 3600).set("contributor_username", "user3"))))
-        .apply(new TopWikipediaSessions.ComputeTopSessions(1.0));
-
-    PAssert.that(output).containsInAnyOrder(Arrays.asList(
-        "user1 : [1970-01-01T00:00:00.000Z..1970-01-01T01:00:02.000Z)"
-        + " : 3 : 1970-01-01T00:00:00.000Z",
-        "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)"
-        + " : 1 : 1970-02-01T00:00:00.000Z"));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoesTest.java
deleted file mode 100644
index aadfa51..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoesTest.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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn;
-import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.FormatCountsFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.List;
-
-/**
- * Test case for {@link BigQueryTornadoes}.
- */
-@RunWith(JUnit4.class)
-public class BigQueryTornadoesTest {
-
-  @Test
-  public void testExtractTornadoes() throws Exception {
-    TableRow row = new TableRow()
-          .set("month", "6")
-          .set("tornado", true);
-    DoFnTester<TableRow, Integer> extractWordsFn =
-        DoFnTester.of(new ExtractTornadoesFn());
-    Assert.assertThat(extractWordsFn.processBatch(row),
-                      CoreMatchers.hasItems(6));
-  }
-
-  @Test
-  public void testNoTornadoes() throws Exception {
-    TableRow row = new TableRow()
-          .set("month", 6)
-          .set("tornado", false);
-    DoFnTester<TableRow, Integer> extractWordsFn =
-        DoFnTester.of(new ExtractTornadoesFn());
-    Assert.assertTrue(extractWordsFn.processBatch(row).isEmpty());
-  }
-
-  @Test
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  public void testFormatCounts() throws Exception {
-    DoFnTester<KV<Integer, Long>, TableRow> formatCountsFn =
-        DoFnTester.of(new FormatCountsFn());
-    KV empty[] = {};
-    List<TableRow> results = formatCountsFn.processBatch(empty);
-    Assert.assertTrue(results.size() == 0);
-    KV input[] = { KV.of(3, 0L),
-                   KV.of(4, Long.MAX_VALUE),
-                   KV.of(5, Long.MIN_VALUE) };
-    results = formatCountsFn.processBatch(input);
-    Assert.assertEquals(results.size(), 3);
-    Assert.assertEquals(results.get(0).get("month"), 3);
-    Assert.assertEquals(results.get(0).get("tornado_count"), 0L);
-    Assert.assertEquals(results.get(1).get("month"), 4);
-    Assert.assertEquals(results.get(1).get("tornado_count"), Long.MAX_VALUE);
-    Assert.assertEquals(results.get(2).get("month"), 5);
-    Assert.assertEquals(results.get(2).get("tornado_count"), Long.MIN_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamplesTest.java
deleted file mode 100644
index 56efe76..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamplesTest.java
+++ /dev/null
@@ -1,91 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn;
-import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.List;
-
-/** Unit tests for {@link CombinePerKeyExamples}. */
-@RunWith(JUnit4.class)
-public class CombinePerKeyExamplesTest {
-
-  private static final TableRow row1 = new TableRow()
-      .set("corpus", "king_lear").set("word", "snuffleupaguses");
-  private static final TableRow row2 = new TableRow()
-      .set("corpus", "macbeth").set("word", "antidisestablishmentarianism");
-  private static final TableRow row3 = new TableRow()
-      .set("corpus", "king_lear").set("word", "antidisestablishmentarianism");
-  private static final TableRow row4 = new TableRow()
-      .set("corpus", "macbeth").set("word", "bob");
-  private static final TableRow row5 = new TableRow()
-      .set("corpus", "king_lear").set("word", "hi");
-
-  static final TableRow[] ROWS_ARRAY = new TableRow[] {
-    row1, row2, row3, row4, row5
-  };
-
-  private static final KV<String, String> tuple1 = KV.of("snuffleupaguses", "king_lear");
-  private static final KV<String, String> tuple2 = KV.of("antidisestablishmentarianism", "macbeth");
-  private static final KV<String, String> tuple3 = KV.of("antidisestablishmentarianism",
-      "king_lear");
-
-  private static final KV<String, String> combinedTuple1 = KV.of("antidisestablishmentarianism",
-      "king_lear,macbeth");
-  private static final KV<String, String> combinedTuple2 = KV.of("snuffleupaguses", "king_lear");
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  static final KV<String, String>[] COMBINED_TUPLES_ARRAY = new KV[] {
-    combinedTuple1, combinedTuple2
-  };
-
-  private static final TableRow resultRow1 = new TableRow()
-      .set("word", "snuffleupaguses").set("all_plays", "king_lear");
-  private static final TableRow resultRow2 = new TableRow()
-      .set("word", "antidisestablishmentarianism")
-      .set("all_plays", "king_lear,macbeth");
-
-  @Test
-  public void testExtractLargeWordsFn() {
-    DoFnTester<TableRow, KV<String, String>> extractLargeWordsFn =
-        DoFnTester.of(new ExtractLargeWordsFn());
-    List<KV<String, String>> results = extractLargeWordsFn.processBatch(ROWS_ARRAY);
-    Assert.assertThat(results, CoreMatchers.hasItem(tuple1));
-    Assert.assertThat(results, CoreMatchers.hasItem(tuple2));
-    Assert.assertThat(results, CoreMatchers.hasItem(tuple3));
-  }
-
-  @Test
-  public void testFormatShakespeareOutputFn() {
-    DoFnTester<KV<String, String>, TableRow> formatShakespeareOutputFn =
-        DoFnTester.of(new FormatShakespeareOutputFn());
-    List<TableRow> results = formatShakespeareOutputFn.processBatch(COMBINED_TUPLES_ARRAY);
-    Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
-    Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/DeDupExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/DeDupExampleTest.java
deleted file mode 100644
index 6e9e3ed..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/DeDupExampleTest.java
+++ /dev/null
@@ -1,84 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/** Unit tests for {@link DeDupExample}. */
-@RunWith(JUnit4.class)
-public class DeDupExampleTest {
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testRemoveDuplicates() {
-    List<String> strings = Arrays.asList(
-        "k1",
-        "k5",
-        "k5",
-        "k2",
-        "k1",
-        "k2",
-        "k3");
-
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input =
-        p.apply(Create.of(strings)
-            .withCoder(StringUtf8Coder.of()));
-
-    PCollection<String> output =
-        input.apply(RemoveDuplicates.<String>create());
-
-    PAssert.that(output)
-        .containsInAnyOrder("k1", "k5", "k2", "k3");
-    p.run();
-  }
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testRemoveDuplicatesEmpty() {
-    List<String> strings = Arrays.asList();
-
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input =
-        p.apply(Create.of(strings)
-            .withCoder(StringUtf8Coder.of()));
-
-    PCollection<String> output =
-        input.apply(RemoveDuplicates.<String>create());
-
-    PAssert.that(output).empty();
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/FilterExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/FilterExamplesTest.java
deleted file mode 100644
index 2bd94d5..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/FilterExamplesTest.java
+++ /dev/null
@@ -1,86 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.FilterExamples.FilterSingleMonthDataFn;
-import com.google.cloud.dataflow.examples.cookbook.FilterExamples.ProjectionFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/** Unit tests for {@link FilterExamples}. */
-@RunWith(JUnit4.class)
-public class FilterExamplesTest {
-
-  private static final TableRow row1 = new TableRow()
-      .set("month", "6").set("day", "21")
-      .set("year", "2014").set("mean_temp", "85.3")
-      .set("tornado", true);
-  private static final TableRow row2 = new TableRow()
-      .set("month", "7").set("day", "20")
-      .set("year", "2014").set("mean_temp", "75.4")
-      .set("tornado", false);
-  private static final TableRow row3 = new TableRow()
-      .set("month", "6").set("day", "18")
-      .set("year", "2014").set("mean_temp", "45.3")
-      .set("tornado", true);
-  static final TableRow[] ROWS_ARRAY = new TableRow[] {
-    row1, row2, row3
-  };
-  static final List<TableRow> ROWS = Arrays.asList(ROWS_ARRAY);
-
-  private static final TableRow outRow1 = new TableRow()
-      .set("year", 2014).set("month", 6)
-      .set("day", 21).set("mean_temp", 85.3);
-  private static final TableRow outRow2 = new TableRow()
-      .set("year", 2014).set("month", 7)
-      .set("day", 20).set("mean_temp", 75.4);
-  private static final TableRow outRow3 = new TableRow()
-      .set("year", 2014).set("month", 6)
-      .set("day", 18).set("mean_temp", 45.3);
-  private static final TableRow[] PROJROWS_ARRAY = new TableRow[] {
-    outRow1, outRow2, outRow3
-  };
-
-
-  @Test
-  public void testProjectionFn() {
-    DoFnTester<TableRow, TableRow> projectionFn =
-        DoFnTester.of(new ProjectionFn());
-    List<TableRow> results = projectionFn.processBatch(ROWS_ARRAY);
-    Assert.assertThat(results, CoreMatchers.hasItem(outRow1));
-    Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
-    Assert.assertThat(results, CoreMatchers.hasItem(outRow3));
-  }
-
-  @Test
-  public void testFilterSingleMonthDataFn() {
-    DoFnTester<TableRow, TableRow> filterSingleMonthDataFn =
-        DoFnTester.of(new FilterSingleMonthDataFn(7));
-    List<TableRow> results = filterSingleMonthDataFn.processBatch(PROJROWS_ARRAY);
-    Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/JoinExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/JoinExamplesTest.java
deleted file mode 100644
index 259ce08..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/JoinExamplesTest.java
+++ /dev/null
@@ -1,115 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractCountryInfoFn;
-import com.google.cloud.dataflow.examples.cookbook.JoinExamples.ExtractEventDataFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/** Unit tests for {@link JoinExamples}. */
-@RunWith(JUnit4.class)
-public class JoinExamplesTest {
-
-  private static final TableRow row1 = new TableRow()
-        .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
-        .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
-  private static final TableRow row2 = new TableRow()
-        .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
-        .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
-  private static final TableRow row3 = new TableRow()
-        .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
-        .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
-  static final TableRow[] EVENTS = new TableRow[] {
-    row1, row2, row3
-  };
-  static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
-
-  private static final KV<String, String> kv1 = KV.of("VM",
-      "Date: 20141212, Actor1: LAOS, url: http://www.chicagotribune.com");
-  private static final KV<String, String> kv2 = KV.of("BE",
-      "Date: 20141213, Actor1: AFGHANISTAN, url: http://cnn.com");
-  private static final KV<String, String> kv3 = KV.of("BE", "Belgium");
-  private static final KV<String, String> kv4 = KV.of("VM", "Vietnam");
-
-  private static final TableRow cc1 = new TableRow()
-        .set("FIPSCC", "VM").set("HumanName", "Vietnam");
-  private static final TableRow cc2 = new TableRow()
-        .set("FIPSCC", "BE").set("HumanName", "Belgium");
-  static final TableRow[] CCS = new TableRow[] {
-    cc1, cc2
-  };
-  static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
-
-  static final String[] JOINED_EVENTS = new String[] {
-      "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
-          + "url: http://www.chicagotribune.com",
-      "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
-          + "url: http://cnn.com",
-      "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
-          + "url: http://cnn.com"
-    };
-
-  @Test
-  public void testExtractEventDataFn() {
-    DoFnTester<TableRow, KV<String, String>> extractEventDataFn =
-        DoFnTester.of(new ExtractEventDataFn());
-    List<KV<String, String>> results = extractEventDataFn.processBatch(EVENTS);
-    Assert.assertThat(results, CoreMatchers.hasItem(kv1));
-    Assert.assertThat(results, CoreMatchers.hasItem(kv2));
-  }
-
-  @Test
-  public void testExtractCountryInfoFn() {
-    DoFnTester<TableRow, KV<String, String>> extractCountryInfoFn =
-        DoFnTester.of(new ExtractCountryInfoFn());
-    List<KV<String, String>> results = extractCountryInfoFn.processBatch(CCS);
-    Assert.assertThat(results, CoreMatchers.hasItem(kv3));
-    Assert.assertThat(results, CoreMatchers.hasItem(kv4));
-  }
-
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testJoin() throws java.lang.Exception {
-    Pipeline p = TestPipeline.create();
-    PCollection<TableRow> input1 = p.apply("CreateEvent", Create.of(EVENT_ARRAY));
-    PCollection<TableRow> input2 = p.apply("CreateCC", Create.of(CC_ARRAY));
-
-    PCollection<String> output = JoinExamples.joinEvents(input1, input2);
-    PAssert.that(output).containsInAnyOrder(JOINED_EVENTS);
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java
deleted file mode 100644
index f3ffcea..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java
+++ /dev/null
@@ -1,86 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.ExtractTempFn;
-import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.List;
-
-/** Unit tests for {@link MaxPerKeyExamples}. */
-@RunWith(JUnit4.class)
-public class MaxPerKeyExamplesTest {
-
-  private static final TableRow row1 = new TableRow()
-        .set("month", "6").set("day", "21")
-        .set("year", "2014").set("mean_temp", "85.3")
-        .set("tornado", true);
-  private static final TableRow row2 = new TableRow()
-        .set("month", "7").set("day", "20")
-        .set("year", "2014").set("mean_temp", "75.4")
-        .set("tornado", false);
-  private static final TableRow row3 = new TableRow()
-        .set("month", "6").set("day", "18")
-        .set("year", "2014").set("mean_temp", "45.3")
-        .set("tornado", true);
-  private static final List<TableRow> TEST_ROWS = ImmutableList.of(row1, row2, row3);
-
-  private static final KV<Integer, Double> kv1 = KV.of(6, 85.3);
-  private static final KV<Integer, Double> kv2 = KV.of(6, 45.3);
-  private static final KV<Integer, Double> kv3 = KV.of(7, 75.4);
-
-  private static final List<KV<Integer, Double>> TEST_KVS = ImmutableList.of(kv1, kv2, kv3);
-
-  private static final TableRow resultRow1 = new TableRow()
-      .set("month", 6)
-      .set("max_mean_temp", 85.3);
-  private static final TableRow resultRow2 = new TableRow()
-      .set("month", 7)
-      .set("max_mean_temp", 75.4);
-
-
-  @Test
-  public void testExtractTempFn() {
-    DoFnTester<TableRow, KV<Integer, Double>> extractTempFn =
-        DoFnTester.of(new ExtractTempFn());
-    List<KV<Integer, Double>> results = extractTempFn.processBatch(TEST_ROWS);
-    Assert.assertThat(results, CoreMatchers.hasItem(kv1));
-    Assert.assertThat(results, CoreMatchers.hasItem(kv2));
-    Assert.assertThat(results, CoreMatchers.hasItem(kv3));
-  }
-
-  @Test
-  public void testFormatMaxesFn() {
-    DoFnTester<KV<Integer, Double>, TableRow> formatMaxesFnFn =
-        DoFnTester.of(new FormatMaxesFn());
-    List<TableRow> results = formatMaxesFnFn.processBatch(TEST_KVS);
-    Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
-    Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/TriggerExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/TriggerExampleTest.java
deleted file mode 100644
index 3664561..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/cookbook/TriggerExampleTest.java
+++ /dev/null
@@ -1,140 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.examples.cookbook.TriggerExample.ExtractFlowInfo;
-import com.google.cloud.dataflow.examples.cookbook.TriggerExample.TotalFlow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Unit Tests for {@link TriggerExample}.
- * The results generated by triggers are by definition non-deterministic and hence hard to test.
- * The unit test does not test all aspects of the example.
- */
-@RunWith(JUnit4.class)
-public class TriggerExampleTest {
-
-  private static final String[] INPUT =
-    {"01/01/2010 00:00:00,1108302,94,E,ML,36,100,29,0.0065,66,9,1,0.001,74.8,1,9,3,0.0028,71,1,9,"
-        + "12,0.0099,67.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,,,0", "01/01/2010 00:00:00,"
-            + "1100333,5,N,FR,9,0,39,,,9,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,,0,,,,"};
-
-  private static final List<TimestampedValue<String>> TIME_STAMPED_INPUT = Arrays.asList(
-      TimestampedValue.of("01/01/2010 00:00:00,1108302,5,W,ML,36,100,30,0.0065,66,9,1,0.001,"
-          + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,87.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,"
-          + ",,0", new Instant(60000)),
-      TimestampedValue.of("01/01/2010 00:00:00,1108302,110,E,ML,36,100,40,0.0065,66,9,1,0.001,"
-          + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,67.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,,"
-          + ",,0", new Instant(1)),
-      TimestampedValue.of("01/01/2010 00:00:00,1108302,110,E,ML,36,100,50,0.0065,66,9,1,"
-          + "0.001,74.8,1,9,3,0.0028,71,1,9,12,0.0099,97.4,1,9,13,0.0121,50.0,1,,,,,0,,,,,0"
-          + ",,,,,0,,,,,0", new Instant(1)));
-
-  private static final TableRow OUT_ROW_1 = new TableRow()
-      .set("trigger_type", "default")
-      .set("freeway", "5").set("total_flow", 30)
-      .set("number_of_records", 1)
-      .set("isFirst", true).set("isLast", true)
-      .set("timing", "ON_TIME")
-      .set("window", "[1970-01-01T00:01:00.000Z..1970-01-01T00:02:00.000Z)");
-
-  private static final TableRow OUT_ROW_2 = new TableRow()
-      .set("trigger_type", "default")
-      .set("freeway", "110").set("total_flow", 90)
-      .set("number_of_records", 2)
-      .set("isFirst", true).set("isLast", true)
-      .set("timing", "ON_TIME")
-      .set("window", "[1970-01-01T00:00:00.000Z..1970-01-01T00:01:00.000Z)");
-
-  @Test
-  public void testExtractTotalFlow() {
-    DoFnTester<String, KV<String, Integer>> extractFlowInfow = DoFnTester
-        .of(new ExtractFlowInfo());
-
-    List<KV<String, Integer>> results = extractFlowInfow.processBatch(INPUT);
-    Assert.assertEquals(results.size(), 1);
-    Assert.assertEquals(results.get(0).getKey(), "94");
-    Assert.assertEquals(results.get(0).getValue(), new Integer(29));
-
-    List<KV<String, Integer>> output = extractFlowInfow.processBatch("");
-    Assert.assertEquals(output.size(), 0);
-  }
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testTotalFlow () {
-    Pipeline pipeline = TestPipeline.create();
-    PCollection<KV<String, Integer>> flow = pipeline
-        .apply(Create.timestamped(TIME_STAMPED_INPUT))
-        .apply(ParDo.of(new ExtractFlowInfo()));
-
-    PCollection<TableRow> totalFlow = flow
-        .apply(Window.<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(1))))
-        .apply(new TotalFlow("default"));
-
-    PCollection<TableRow> results =  totalFlow.apply(ParDo.of(new FormatResults()));
-
-
-    PAssert.that(results).containsInAnyOrder(OUT_ROW_1, OUT_ROW_2);
-    pipeline.run();
-
-  }
-
-  static class FormatResults extends DoFn<TableRow, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      TableRow element = c.element();
-      TableRow row = new TableRow()
-          .set("trigger_type", element.get("trigger_type"))
-          .set("freeway", element.get("freeway"))
-          .set("total_flow", element.get("total_flow"))
-          .set("number_of_records", element.get("number_of_records"))
-          .set("isFirst", element.get("isFirst"))
-          .set("isLast", element.get("isLast"))
-          .set("timing", element.get("timing"))
-          .set("window", element.get("window"));
-      c.output(row);
-    }
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
new file mode 100644
index 0000000..f9e4a2d
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.common.io.Files;
+
+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;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Tests for {@link DebuggingWordCount}.
+ */
+@RunWith(JUnit4.class)
+public class DebuggingWordCountTest {
+  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testDebuggingWordCount() throws Exception {
+    File file = tmpFolder.newFile();
+    Files.write("stomach secret Flourish message Flourish here Flourish", file,
+        StandardCharsets.UTF_8);
+    DebuggingWordCount.main(new String[]{"--inputFile=" + file.getAbsolutePath()});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
new file mode 100644
index 0000000..b1f4f27
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.examples.WordCount.CountWords;
+import com.google.cloud.dataflow.examples.WordCount.ExtractWordsFn;
+import com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tests of WordCount.
+ */
+@RunWith(JUnit4.class)
+public class WordCountTest {
+
+  /** Example test that tests a specific DoFn. */
+  @Test
+  public void testExtractWordsFn() {
+    DoFnTester<String, String> extractWordsFn =
+        DoFnTester.of(new ExtractWordsFn());
+
+    Assert.assertThat(extractWordsFn.processBatch(" some  input  words "),
+                      CoreMatchers.hasItems("some", "input", "words"));
+    Assert.assertThat(extractWordsFn.processBatch(" "),
+                      CoreMatchers.<String>hasItems());
+    Assert.assertThat(extractWordsFn.processBatch(" some ", " input", " words"),
+                      CoreMatchers.hasItems("some", "input", "words"));
+  }
+
+  static final String[] WORDS_ARRAY = new String[] {
+    "hi there", "hi", "hi sue bob",
+    "hi sue", "", "bob hi"};
+
+  static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+
+  static final String[] COUNTS_ARRAY = new String[] {
+      "hi: 5", "there: 1", "sue: 2", "bob: 2"};
+
+  /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testCountWords() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()));
+
+    PCollection<String> output = input.apply(new CountWords())
+      .apply(MapElements.via(new FormatAsTextFn()));
+
+    PAssert.that(output).containsInAnyOrder(COUNTS_ARRAY);
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
new file mode 100644
index 0000000..a70aee1
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.cloud.dataflow.examples.complete.AutoComplete.CompletionCandidate;
+import com.google.cloud.dataflow.examples.complete.AutoComplete.ComputeTopCompletions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Tests of AutoComplete.
+ */
+@RunWith(Parameterized.class)
+public class AutoCompleteTest implements Serializable {
+  private boolean recursive;
+
+  public AutoCompleteTest(Boolean recursive) {
+    this.recursive = recursive;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> testRecursive() {
+    return Arrays.asList(new Object[][] {
+        { true },
+        { false }
+      });
+  }
+
+  @Test
+  public void testAutoComplete() {
+    List<String> words = Arrays.asList(
+        "apple",
+        "apple",
+        "apricot",
+        "banana",
+        "blackberry",
+        "blackberry",
+        "blackberry",
+        "blueberry",
+        "blueberry",
+        "cherry");
+
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(words));
+
+    PCollection<KV<String, List<CompletionCandidate>>> output =
+      input.apply(new ComputeTopCompletions(2, recursive))
+           .apply(Filter.byPredicate(
+                        new SerializableFunction<KV<String, List<CompletionCandidate>>, Boolean>() {
+                          @Override
+                          public Boolean apply(KV<String, List<CompletionCandidate>> element) {
+                            return element.getKey().length() <= 2;
+                          }
+                      }));
+
+    PAssert.that(output).containsInAnyOrder(
+        KV.of("a", parseList("apple:2", "apricot:1")),
+        KV.of("ap", parseList("apple:2", "apricot:1")),
+        KV.of("b", parseList("blackberry:3", "blueberry:2")),
+        KV.of("ba", parseList("banana:1")),
+        KV.of("bl", parseList("blackberry:3", "blueberry:2")),
+        KV.of("c", parseList("cherry:1")),
+        KV.of("ch", parseList("cherry:1")));
+    p.run();
+  }
+
+  @Test
+  public void testTinyAutoComplete() {
+    List<String> words = Arrays.asList("x", "x", "x", "xy", "xy", "xyz");
+
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of(words));
+
+    PCollection<KV<String, List<CompletionCandidate>>> output =
+      input.apply(new ComputeTopCompletions(2, recursive));
+
+    PAssert.that(output).containsInAnyOrder(
+        KV.of("x", parseList("x:3", "xy:2")),
+        KV.of("xy", parseList("xy:2", "xyz:1")),
+        KV.of("xyz", parseList("xyz:1")));
+    p.run();
+  }
+
+  @Test
+  public void testWindowedAutoComplete() {
+    List<TimestampedValue<String>> words = Arrays.asList(
+        TimestampedValue.of("xA", new Instant(1)),
+        TimestampedValue.of("xA", new Instant(1)),
+        TimestampedValue.of("xB", new Instant(1)),
+        TimestampedValue.of("xB", new Instant(2)),
+        TimestampedValue.of("xB", new Instant(2)));
+
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input = p
+      .apply(Create.of(words))
+      .apply(new ReifyTimestamps<String>());
+
+    PCollection<KV<String, List<CompletionCandidate>>> output =
+      input.apply(Window.<String>into(SlidingWindows.of(new Duration(2))))
+           .apply(new ComputeTopCompletions(2, recursive));
+
+    PAssert.that(output).containsInAnyOrder(
+        // Window [0, 2)
+        KV.of("x", parseList("xA:2", "xB:1")),
+        KV.of("xA", parseList("xA:2")),
+        KV.of("xB", parseList("xB:1")),
+
+        // Window [1, 3)
+        KV.of("x", parseList("xB:3", "xA:2")),
+        KV.of("xA", parseList("xA:2")),
+        KV.of("xB", parseList("xB:3")),
+
+        // Window [2, 3)
+        KV.of("x", parseList("xB:2")),
+        KV.of("xB", parseList("xB:2")));
+    p.run();
+  }
+
+  private static List<CompletionCandidate> parseList(String... entries) {
+    List<CompletionCandidate> all = new ArrayList<>();
+    for (String s : entries) {
+      String[] countValue = s.split(":");
+      all.add(new CompletionCandidate(countValue[0], Integer.valueOf(countValue[1])));
+    }
+    return all;
+  }
+
+  private static class ReifyTimestamps<T>
+      extends PTransform<PCollection<TimestampedValue<T>>, PCollection<T>> {
+    @Override
+    public PCollection<T> apply(PCollection<TimestampedValue<T>> input) {
+      return input.apply(ParDo.of(new DoFn<TimestampedValue<T>, T>() {
+        @Override
+        public void processElement(ProcessContext c) {
+          c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp());
+        }
+      }));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
new file mode 100644
index 0000000..5989ce8
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Keys;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.net.URI;
+import java.util.Arrays;
+
+/**
+ * Tests of {@link TfIdf}.
+ */
+@RunWith(JUnit4.class)
+public class TfIdfTest {
+
+  /** Test that the example runs. */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testTfIdf() throws Exception {
+    Pipeline pipeline = TestPipeline.create();
+
+    pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+    PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
+        .apply(Create.of(
+            KV.of(new URI("x"), "a b c d"),
+            KV.of(new URI("y"), "a b c"),
+            KV.of(new URI("z"), "a m n")))
+        .apply(new TfIdf.ComputeTfIdf());
+
+    PCollection<String> words = wordToUriAndTfIdf
+        .apply(Keys.<String>create())
+        .apply(RemoveDuplicates.<String>create());
+
+    PAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d"));
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
new file mode 100644
index 0000000..52f69b0
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+
+/** Unit tests for {@link TopWikipediaSessions}. */
+@RunWith(JUnit4.class)
+public class TopWikipediaSessionsTest {
+  @Test
+  @Category(RunnableOnService.class)
+  public void testComputeTopUsers() {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> output =
+        p.apply(Create.of(Arrays.asList(
+            new TableRow().set("timestamp", 0).set("contributor_username", "user1"),
+            new TableRow().set("timestamp", 1).set("contributor_username", "user1"),
+            new TableRow().set("timestamp", 2).set("contributor_username", "user1"),
+            new TableRow().set("timestamp", 0).set("contributor_username", "user2"),
+            new TableRow().set("timestamp", 1).set("contributor_username", "user2"),
+            new TableRow().set("timestamp", 3601).set("contributor_username", "user2"),
+            new TableRow().set("timestamp", 3602).set("contributor_username", "user2"),
+            new TableRow().set("timestamp", 35 * 24 * 3600).set("contributor_username", "user3"))))
+        .apply(new TopWikipediaSessions.ComputeTopSessions(1.0));
+
+    PAssert.that(output).containsInAnyOrder(Arrays.asList(
+        "user1 : [1970-01-01T00:00:00.000Z..1970-01-01T01:00:02.000Z)"
+        + " : 3 : 1970-01-01T00:00:00.000Z",
+        "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)"
+        + " : 1 : 1970-02-01T00:00:00.000Z"));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
new file mode 100644
index 0000000..aadfa51
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn;
+import com.google.cloud.dataflow.examples.cookbook.BigQueryTornadoes.FormatCountsFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.values.KV;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.List;
+
+/**
+ * Test case for {@link BigQueryTornadoes}.
+ */
+@RunWith(JUnit4.class)
+public class BigQueryTornadoesTest {
+
+  @Test
+  public void testExtractTornadoes() throws Exception {
+    TableRow row = new TableRow()
+          .set("month", "6")
+          .set("tornado", true);
+    DoFnTester<TableRow, Integer> extractWordsFn =
+        DoFnTester.of(new ExtractTornadoesFn());
+    Assert.assertThat(extractWordsFn.processBatch(row),
+                      CoreMatchers.hasItems(6));
+  }
+
+  @Test
+  public void testNoTornadoes() throws Exception {
+    TableRow row = new TableRow()
+          .set("month", 6)
+          .set("tornado", false);
+    DoFnTester<TableRow, Integer> extractWordsFn =
+        DoFnTester.of(new ExtractTornadoesFn());
+    Assert.assertTrue(extractWordsFn.processBatch(row).isEmpty());
+  }
+
+  @Test
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  public void testFormatCounts() throws Exception {
+    DoFnTester<KV<Integer, Long>, TableRow> formatCountsFn =
+        DoFnTester.of(new FormatCountsFn());
+    KV empty[] = {};
+    List<TableRow> results = formatCountsFn.processBatch(empty);
+    Assert.assertTrue(results.size() == 0);
+    KV input[] = { KV.of(3, 0L),
+                   KV.of(4, Long.MAX_VALUE),
+                   KV.of(5, Long.MIN_VALUE) };
+    results = formatCountsFn.processBatch(input);
+    Assert.assertEquals(results.size(), 3);
+    Assert.assertEquals(results.get(0).get("month"), 3);
+    Assert.assertEquals(results.get(0).get("tornado_count"), 0L);
+    Assert.assertEquals(results.get(1).get("month"), 4);
+    Assert.assertEquals(results.get(1).get("tornado_count"), Long.MAX_VALUE);
+    Assert.assertEquals(results.get(2).get("month"), 5);
+    Assert.assertEquals(results.get(2).get("tornado_count"), Long.MIN_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
new file mode 100644
index 0000000..56efe76
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn;
+import com.google.cloud.dataflow.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.values.KV;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.List;
+
+/** Unit tests for {@link CombinePerKeyExamples}. */
+@RunWith(JUnit4.class)
+public class CombinePerKeyExamplesTest {
+
+  private static final TableRow row1 = new TableRow()
+      .set("corpus", "king_lear").set("word", "snuffleupaguses");
+  private static final TableRow row2 = new TableRow()
+      .set("corpus", "macbeth").set("word", "antidisestablishmentarianism");
+  private static final TableRow row3 = new TableRow()
+      .set("corpus", "king_lear").set("word", "antidisestablishmentarianism");
+  private static final TableRow row4 = new TableRow()
+      .set("corpus", "macbeth").set("word", "bob");
+  private static final TableRow row5 = new TableRow()
+      .set("corpus", "king_lear").set("word", "hi");
+
+  static final TableRow[] ROWS_ARRAY = new TableRow[] {
+    row1, row2, row3, row4, row5
+  };
+
+  private static final KV<String, String> tuple1 = KV.of("snuffleupaguses", "king_lear");
+  private static final KV<String, String> tuple2 = KV.of("antidisestablishmentarianism", "macbeth");
+  private static final KV<String, String> tuple3 = KV.of("antidisestablishmentarianism",
+      "king_lear");
+
+  private static final KV<String, String> combinedTuple1 = KV.of("antidisestablishmentarianism",
+      "king_lear,macbeth");
+  private static final KV<String, String> combinedTuple2 = KV.of("snuffleupaguses", "king_lear");
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  static final KV<String, String>[] COMBINED_TUPLES_ARRAY = new KV[] {
+    combinedTuple1, combinedTuple2
+  };
+
+  private static final TableRow resultRow1 = new TableRow()
+      .set("word", "snuffleupaguses").set("all_plays", "king_lear");
+  private static final TableRow resultRow2 = new TableRow()
+      .set("word", "antidisestablishmentarianism")
+      .set("all_plays", "king_lear,macbeth");
+
+  @Test
+  public void testExtractLargeWordsFn() {
+    DoFnTester<TableRow, KV<String, String>> extractLargeWordsFn =
+        DoFnTester.of(new ExtractLargeWordsFn());
+    List<KV<String, String>> results = extractLargeWordsFn.processBatch(ROWS_ARRAY);
+    Assert.assertThat(results, CoreMatchers.hasItem(tuple1));
+    Assert.assertThat(results, CoreMatchers.hasItem(tuple2));
+    Assert.assertThat(results, CoreMatchers.hasItem(tuple3));
+  }
+
+  @Test
+  public void testFormatShakespeareOutputFn() {
+    DoFnTester<KV<String, String>, TableRow> formatShakespeareOutputFn =
+        DoFnTester.of(new FormatShakespeareOutputFn());
+    List<TableRow> results = formatShakespeareOutputFn.processBatch(COMBINED_TUPLES_ARRAY);
+    Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
+    Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
new file mode 100644
index 0000000..6e9e3ed
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/** Unit tests for {@link DeDupExample}. */
+@RunWith(JUnit4.class)
+public class DeDupExampleTest {
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testRemoveDuplicates() {
+    List<String> strings = Arrays.asList(
+        "k1",
+        "k5",
+        "k5",
+        "k2",
+        "k1",
+        "k2",
+        "k3");
+
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input =
+        p.apply(Create.of(strings)
+            .withCoder(StringUtf8Coder.of()));
+
+    PCollection<String> output =
+        input.apply(RemoveDuplicates.<String>create());
+
+    PAssert.that(output)
+        .containsInAnyOrder("k1", "k5", "k2", "k3");
+    p.run();
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testRemoveDuplicatesEmpty() {
+    List<String> strings = Arrays.asList();
+
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> input =
+        p.apply(Create.of(strings)
+            .withCoder(StringUtf8Coder.of()));
+
+    PCollection<String> output =
+        input.apply(RemoveDuplicates.<String>create());
+
+    PAssert.that(output).empty();
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
new file mode 100644
index 0000000..2bd94d5
--- /dev/null
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.examples.cookbook.FilterExamples.FilterSingleMonthDataFn;
+import com.google.cloud.dataflow.examples.cookbook.FilterExamples.ProjectionFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/** Unit tests for {@link FilterExamples}. */
+@RunWith(JUnit4.class)
+public class FilterExamplesTest {
+
+  private static final TableRow row1 = new TableRow()
+      .set("month", "6").set("day", "21")
+      .set("year", "2014").set("mean_temp", "85.3")
+      .set("tornado", true);
+  private static final TableRow row2 = new TableRow()
+      .set("month", "7").set("day", "20")
+      .set("year", "2014").set("mean_temp", "75.4")
+      .set("tornado", false);
+  private static final TableRow row3 = new TableRow()
+      .set("month", "6").set("day", "18")
+      .set("year", "2014").set("mean_temp", "45.3")
+      .set("tornado", true);
+  static final TableRow[] ROWS_ARRAY = new TableRow[] {
+    row1, row2, row3
+  };
+  static final List<TableRow> ROWS = Arrays.asList(ROWS_ARRAY);
+
+  private static final TableRow outRow1 = new TableRow()
+      .set("year", 2014).set("month", 6)
+      .set("day", 21).set("mean_temp", 85.3);
+  private static final TableRow outRow2 = new TableRow()
+      .set("year", 2014).set("month", 7)
+      .set("day", 20).set("mean_temp", 75.4);
+  private static final TableRow outRow3 = new TableRow()
+      .set("year", 2014).set("month", 6)
+      .set("day", 18).set("mean_temp", 45.3);
+  private static final TableRow[] PROJROWS_ARRAY = new TableRow[] {
+    outRow1, outRow2, outRow3
+  };
+
+
+  @Test
+  public void testProjectionFn() {
+    DoFnTester<TableRow, TableRow> projectionFn =
+        DoFnTester.of(new ProjectionFn());
+    List<TableRow> results = projectionFn.processBatch(ROWS_ARRAY);
+    Assert.assertThat(results, CoreMatchers.hasItem(outRow1));
+    Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
+    Assert.assertThat(results, CoreMatchers.hasItem(outRow3));
+  }
+
+  @Test
+  public void testFilterSingleMonthDataFn() {
+    DoFnTester<TableRow, TableRow> filterSingleMonthDataFn =
+        DoFnTester.of(new FilterSingleMonthDataFn(7));
+    List<TableRow> results = filterSingleMonthDataFn.processBatch(PROJROWS_ARRAY);
+    Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
+  }
+}


[31/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
new file mode 100644
index 0000000..8f8c653
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
@@ -0,0 +1,39 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.services.dataflow.model.Environment;
+
+/**
+ * An instance of this class can be passed to the
+ * {@link DataflowPipelineRunner} to add user defined hooks to be
+ * invoked at various times during pipeline execution.
+ */
+@Experimental
+public class DataflowPipelineRunnerHooks {
+  /**
+   * Allows the user to modify the environment of their job before their job is submitted
+   * to the service for execution.
+   *
+   * @param environment The environment of the job. Users can make change to this instance in order
+   *     to change the environment with which their job executes on the service.
+   */
+  public void modifyEnvironmentBeforeSubmission(Environment environment) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
new file mode 100644
index 0000000..fa7067e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
@@ -0,0 +1,1100 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
+import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
+import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString;
+import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray;
+import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
+import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
+import static com.google.cloud.dataflow.sdk.util.Structs.addList;
+import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
+import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
+import static com.google.cloud.dataflow.sdk.util.Structs.addString;
+import static com.google.cloud.dataflow.sdk.util.Structs.getString;
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.services.dataflow.model.AutoscalingSettings;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Disk;
+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.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderException;
+import com.google.cloud.dataflow.sdk.coders.IterableCoder;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.StreamingOptions;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
+import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator;
+import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
+import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
+import com.google.cloud.dataflow.sdk.util.CloudObject;
+import com.google.cloud.dataflow.sdk.util.DoFnInfo;
+import com.google.cloud.dataflow.sdk.util.OutputReference;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.SerializableUtils;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects
+ * into Cloud Dataflow Service API {@link Job}s.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class DataflowPipelineTranslator {
+  // Must be kept in sync with their internal counterparts.
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+
+  /**
+   * A map from {@link PTransform} subclass to the corresponding
+   * {@link TransformTranslator} to use to translate that transform.
+   *
+   * <p>A static map that contains system-wide defaults.
+   */
+  private static Map<Class, TransformTranslator> transformTranslators =
+      new HashMap<>();
+
+  /** Provided configuration options. */
+  private final DataflowPipelineOptions options;
+
+  /**
+   * Constructs a translator from the provided options.
+   *
+   * @param options Properties that configure the translator.
+   *
+   * @return The newly created translator.
+   */
+  public static DataflowPipelineTranslator fromOptions(
+      DataflowPipelineOptions options) {
+    return new DataflowPipelineTranslator(options);
+  }
+
+  private DataflowPipelineTranslator(DataflowPipelineOptions options) {
+    this.options = options;
+  }
+
+  /**
+   * Translates a {@link Pipeline} into a {@code JobSpecification}.
+   */
+  public JobSpecification translate(
+      Pipeline pipeline,
+      DataflowPipelineRunner runner,
+      List<DataflowPackage> packages) {
+
+    Translator translator = new Translator(pipeline, runner);
+    Job result = translator.translate(packages);
+    return new JobSpecification(result, Collections.unmodifiableMap(translator.stepNames));
+  }
+
+  /**
+   * The result of a job translation.
+   *
+   * <p>Used to pass the result {@link Job} and any state that was used to construct the job that
+   * may be of use to other classes (eg the {@link PTransform} to StepName mapping).
+   */
+  public static class JobSpecification {
+    private final Job job;
+    private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+
+    public JobSpecification(Job job, Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
+      this.job = job;
+      this.stepNames = stepNames;
+    }
+
+    public Job getJob() {
+      return job;
+    }
+
+    /**
+     * Returns the mapping of {@link AppliedPTransform AppliedPTransforms} to the internal step
+     * name for that {@code AppliedPTransform}.
+     */
+    public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
+      return stepNames;
+    }
+  }
+
+  /**
+   * Renders a {@link Job} as a string.
+   */
+  public static String jobToString(Job job) {
+    try {
+      return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(job);
+    } catch (JsonProcessingException exc) {
+      throw new IllegalStateException("Failed to render Job as String.", exc);
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Records that instances of the specified PTransform class
+   * should be translated by default by the corresponding
+   * {@link TransformTranslator}.
+   */
+  public static <TransformT extends PTransform> void registerTransformTranslator(
+      Class<TransformT> transformClass,
+      TransformTranslator<? extends TransformT> transformTranslator) {
+    if (transformTranslators.put(transformClass, transformTranslator) != null) {
+      throw new IllegalArgumentException(
+          "defining multiple translators for " + transformClass);
+    }
+  }
+
+  /**
+   * Returns the {@link TransformTranslator} to use for instances of the
+   * specified PTransform class, or null if none registered.
+   */
+  public <TransformT extends PTransform>
+      TransformTranslator<TransformT> getTransformTranslator(Class<TransformT> transformClass) {
+    return transformTranslators.get(transformClass);
+  }
+
+  /**
+   * A {@link TransformTranslator} knows how to translate
+   * a particular subclass of {@link PTransform} for the
+   * Cloud Dataflow service. It does so by
+   * mutating the {@link TranslationContext}.
+   */
+  public interface TransformTranslator<TransformT extends PTransform> {
+    public void translate(TransformT transform,
+                          TranslationContext context);
+  }
+
+  /**
+   * The interface provided to registered callbacks for interacting
+   * with the {@link DataflowPipelineRunner}, including reading and writing the
+   * values of {@link PCollection}s and side inputs ({@link PCollectionView}s).
+   */
+  public interface TranslationContext {
+    /**
+     * Returns the configured pipeline options.
+     */
+    DataflowPipelineOptions getPipelineOptions();
+
+    /**
+     * Returns the input of the currently being translated transform.
+     */
+    <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
+
+    /**
+     * Returns the output of the currently being translated transform.
+     */
+    <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
+
+    /**
+     * Returns the full name of the currently being translated transform.
+     */
+    String getFullName(PTransform<?, ?> transform);
+
+    /**
+     * Adds a step to the Dataflow workflow for the given transform, with
+     * the given Dataflow step type.
+     * This step becomes "current" for the purpose of {@link #addInput} and
+     * {@link #addOutput}.
+     */
+    public void addStep(PTransform<?, ?> transform, String type);
+
+    /**
+     * Adds a pre-defined step to the Dataflow workflow. The given PTransform should be
+     * consistent with the Step, in terms of input, output and coder types.
+     *
+     * <p>This is a low-level operation, when using this method it is up to
+     * the caller to ensure that names do not collide.
+     */
+    public void addStep(PTransform<?, ? extends PValue> transform, Step step);
+
+    /**
+     * Sets the encoding for the current Dataflow step.
+     */
+    public void addEncodingInput(Coder<?> value);
+
+    /**
+     * Adds an input with the given name and value to the current
+     * Dataflow step.
+     */
+    public void addInput(String name, Boolean value);
+
+    /**
+     * Adds an input with the given name and value to the current
+     * Dataflow step.
+     */
+    public void addInput(String name, String value);
+
+    /**
+     * Adds an input with the given name and value to the current
+     * Dataflow step.
+     */
+    public void addInput(String name, Long value);
+
+    /**
+     * Adds an input with the given name to the previously added Dataflow
+     * step, coming from the specified input PValue.
+     */
+    public void addInput(String name, PInput value);
+
+    /**
+     * Adds an input that is a dictionary of strings to objects.
+     */
+    public void addInput(String name, Map<String, Object> elements);
+
+    /**
+     * Adds an input that is a list of objects.
+     */
+    public void addInput(String name, List<? extends Map<String, Object>> elements);
+
+    /**
+     * Adds an output with the given name to the previously added
+     * Dataflow step, producing the specified output {@code PValue},
+     * including its {@code Coder} if a {@code TypedPValue}.  If the
+     * {@code PValue} is a {@code PCollection}, wraps its coder inside
+     * a {@code WindowedValueCoder}.
+     */
+    public void addOutput(String name, PValue value);
+
+    /**
+     * Adds an output with the given name to the previously added
+     * Dataflow step, producing the specified output {@code PValue},
+     * including its {@code Coder} if a {@code TypedPValue}.  If the
+     * {@code PValue} is a {@code PCollection}, wraps its coder inside
+     * a {@code ValueOnlyCoder}.
+     */
+    public void addValueOnlyOutput(String name, PValue value);
+
+    /**
+     * Adds an output with the given name to the previously added
+     * CollectionToSingleton Dataflow step, consuming the specified
+     * input {@code PValue} and producing the specified output
+     * {@code PValue}.  This step requires special treatment for its
+     * output encoding.
+     */
+    public void addCollectionToSingletonOutput(String name,
+                                               PValue inputValue,
+                                               PValue outputValue);
+
+    /**
+     * Encode a PValue reference as an output reference.
+     */
+    public OutputReference asOutputReference(PValue value);
+  }
+
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Translates a Pipeline into the Dataflow representation.
+   */
+  class Translator implements PipelineVisitor, TranslationContext {
+    /** The Pipeline to translate. */
+    private final Pipeline pipeline;
+
+    /** The runner which will execute the pipeline. */
+    private final DataflowPipelineRunner runner;
+
+    /** The Cloud Dataflow Job representation. */
+    private final Job job = new Job();
+
+    /**
+     * Translator is stateful, as addProperty calls refer to the current step.
+     */
+    private Step currentStep;
+
+    /**
+     * A Map from AppliedPTransform to their unique Dataflow step names.
+     */
+    private final Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
+
+    /**
+     * A Map from PValues to their output names used by their producer
+     * Dataflow steps.
+     */
+    private final Map<POutput, String> outputNames = new HashMap<>();
+
+    /**
+     * A Map from PValues to the Coders used for them.
+     */
+    private final Map<POutput, Coder<?>> outputCoders = new HashMap<>();
+
+    /**
+     * The transform currently being applied.
+     */
+    private AppliedPTransform<?, ?, ?> currentTransform;
+
+    /**
+     * Constructs a Translator that will translate the specified
+     * Pipeline into Dataflow objects.
+     */
+    public Translator(Pipeline pipeline, DataflowPipelineRunner runner) {
+      this.pipeline = pipeline;
+      this.runner = runner;
+    }
+
+    /**
+     * Translates this Translator's pipeline onto its writer.
+     * @return a Job definition filled in with the type of job, the environment,
+     * and the job steps.
+     */
+    public Job translate(List<DataflowPackage> packages) {
+      job.setName(options.getJobName().toLowerCase());
+
+      Environment environment = new Environment();
+      job.setEnvironment(environment);
+
+      try {
+        environment.setSdkPipelineOptions(
+            MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class));
+      } catch (IOException e) {
+        throw new IllegalArgumentException(
+            "PipelineOptions specified failed to serialize to JSON.", e);
+      }
+
+      WorkerPool workerPool = new WorkerPool();
+
+      if (options.getTeardownPolicy() != null) {
+        workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName());
+      }
+
+      if (options.isStreaming()) {
+        job.setType("JOB_TYPE_STREAMING");
+      } else {
+        job.setType("JOB_TYPE_BATCH");
+        workerPool.setDiskType(options.getWorkerDiskType());
+      }
+
+      if (options.getWorkerMachineType() != null) {
+        workerPool.setMachineType(options.getWorkerMachineType());
+      }
+
+      workerPool.setPackages(packages);
+      workerPool.setNumWorkers(options.getNumWorkers());
+
+      if (options.isStreaming()) {
+        // Use separate data disk for streaming.
+        Disk disk = new Disk();
+        disk.setDiskType(options.getWorkerDiskType());
+        workerPool.setDataDisks(Collections.singletonList(disk));
+      }
+      if (!Strings.isNullOrEmpty(options.getZone())) {
+        workerPool.setZone(options.getZone());
+      }
+      if (!Strings.isNullOrEmpty(options.getNetwork())) {
+        workerPool.setNetwork(options.getNetwork());
+      }
+      if (!Strings.isNullOrEmpty(options.getSubnetwork())) {
+        workerPool.setSubnetwork(options.getSubnetwork());
+      }
+      if (options.getDiskSizeGb() > 0) {
+        workerPool.setDiskSizeGb(options.getDiskSizeGb());
+      }
+      AutoscalingSettings settings = new AutoscalingSettings();
+      if (options.getAutoscalingAlgorithm() != null) {
+        settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm());
+      }
+      settings.setMaxNumWorkers(options.getMaxNumWorkers());
+      workerPool.setAutoscalingSettings(settings);
+
+      List<WorkerPool> workerPools = new LinkedList<>();
+
+      workerPools.add(workerPool);
+      environment.setWorkerPools(workerPools);
+
+      pipeline.traverseTopologically(this);
+      return job;
+    }
+
+    @Override
+    public DataflowPipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    @Override
+    public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
+      return (InputT) getCurrentTransform(transform).getInput();
+    }
+
+    @Override
+    public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
+      return (OutputT) getCurrentTransform(transform).getOutput();
+    }
+
+    @Override
+    public String getFullName(PTransform<?, ?> transform) {
+      return getCurrentTransform(transform).getFullName();
+    }
+
+    private AppliedPTransform<?, ?, ?> getCurrentTransform(PTransform<?, ?> transform) {
+      checkArgument(
+          currentTransform != null && currentTransform.getTransform() == transform,
+          "can only be called with current transform");
+      return currentTransform;
+    }
+
+    @Override
+    public void enterCompositeTransform(TransformTreeNode node) {
+    }
+
+    @Override
+    public void leaveCompositeTransform(TransformTreeNode node) {
+    }
+
+    @Override
+    public void visitTransform(TransformTreeNode node) {
+      PTransform<?, ?> transform = node.getTransform();
+      TransformTranslator translator =
+          getTransformTranslator(transform.getClass());
+      if (translator == null) {
+        throw new IllegalStateException(
+            "no translator registered for " + transform);
+      }
+      LOG.debug("Translating {}", transform);
+      currentTransform = AppliedPTransform.of(
+          node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
+      translator.translate(transform, this);
+      currentTransform = null;
+    }
+
+    @Override
+    public void visitValue(PValue value, TransformTreeNode producer) {
+      LOG.debug("Checking translation of {}", value);
+      if (value.getProducingTransformInternal() == null) {
+        throw new RuntimeException(
+            "internal error: expecting a PValue "
+            + "to have a producingTransform");
+      }
+      if (!producer.isCompositeNode()) {
+        // Primitive transforms are the only ones assigned step names.
+        asOutputReference(value);
+      }
+    }
+
+    @Override
+    public void addStep(PTransform<?, ?> transform, String type) {
+      String stepName = genStepName();
+      if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
+        throw new IllegalArgumentException(
+            transform + " already has a name specified");
+      }
+      // Start the next "steps" list item.
+      List<Step> steps = job.getSteps();
+      if (steps == null) {
+        steps = new LinkedList<>();
+        job.setSteps(steps);
+      }
+
+      currentStep = new Step();
+      currentStep.setName(stepName);
+      currentStep.setKind(type);
+      steps.add(currentStep);
+      addInput(PropertyNames.USER_NAME, getFullName(transform));
+      addDisplayData(PropertyNames.DISPLAY_DATA, DisplayData.from(transform));
+    }
+
+    @Override
+    public void addStep(PTransform<?, ? extends PValue> transform, Step original) {
+      Step step = original.clone();
+      String stepName = step.getName();
+      if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
+        throw new IllegalArgumentException(transform + " already has a name specified");
+      }
+
+      Map<String, Object> properties = step.getProperties();
+      if (properties != null) {
+        @Nullable List<Map<String, Object>> outputInfoList = null;
+        try {
+          // TODO: This should be done via a Structs accessor.
+          @Nullable List<Map<String, Object>> list =
+              (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
+          outputInfoList = list;
+        } catch (Exception e) {
+          throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
+        }
+        if (outputInfoList != null && outputInfoList.size() > 0) {
+          Map<String, Object> firstOutputPort = outputInfoList.get(0);
+          @Nullable String name;
+          try {
+            name = getString(firstOutputPort, PropertyNames.OUTPUT_NAME);
+          } catch (Exception e) {
+            name = null;
+          }
+          if (name != null) {
+            registerOutputName(getOutput(transform), name);
+          }
+        }
+      }
+
+      List<Step> steps = job.getSteps();
+      if (steps == null) {
+        steps = new LinkedList<>();
+        job.setSteps(steps);
+      }
+      currentStep = step;
+      steps.add(step);
+    }
+
+    @Override
+    public void addEncodingInput(Coder<?> coder) {
+      CloudObject encoding = SerializableUtils.ensureSerializable(coder);
+      addObject(getProperties(), PropertyNames.ENCODING, encoding);
+    }
+
+    @Override
+    public void addInput(String name, Boolean value) {
+      addBoolean(getProperties(), name, value);
+    }
+
+    @Override
+    public void addInput(String name, String value) {
+      addString(getProperties(), name, value);
+    }
+
+    @Override
+    public void addInput(String name, Long value) {
+      addLong(getProperties(), name, value);
+    }
+
+    @Override
+    public void addInput(String name, Map<String, Object> elements) {
+      addDictionary(getProperties(), name, elements);
+    }
+
+    @Override
+    public void addInput(String name, List<? extends Map<String, Object>> elements) {
+      addList(getProperties(), name, elements);
+    }
+
+    @Override
+    public void addInput(String name, PInput value) {
+      if (value instanceof PValue) {
+        addInput(name, asOutputReference((PValue) value));
+      } else {
+        throw new IllegalStateException("Input must be a PValue");
+      }
+    }
+
+    @Override
+    public void addOutput(String name, PValue value) {
+      Coder<?> coder;
+      if (value instanceof TypedPValue) {
+        coder = ((TypedPValue<?>) value).getCoder();
+        if (value instanceof PCollection) {
+          // Wrap the PCollection element Coder inside a WindowedValueCoder.
+          coder = WindowedValue.getFullCoder(
+              coder,
+              ((PCollection<?>) value).getWindowingStrategy().getWindowFn().windowCoder());
+        }
+      } else {
+        // No output coder to encode.
+        coder = null;
+      }
+      addOutput(name, value, coder);
+    }
+
+    @Override
+    public void addValueOnlyOutput(String name, PValue value) {
+      Coder<?> coder;
+      if (value instanceof TypedPValue) {
+        coder = ((TypedPValue<?>) value).getCoder();
+        if (value instanceof PCollection) {
+          // Wrap the PCollection element Coder inside a ValueOnly
+          // WindowedValueCoder.
+          coder = WindowedValue.getValueOnlyCoder(coder);
+        }
+      } else {
+        // No output coder to encode.
+        coder = null;
+      }
+      addOutput(name, value, coder);
+    }
+
+    @Override
+    public void addCollectionToSingletonOutput(String name,
+                                               PValue inputValue,
+                                               PValue outputValue) {
+      Coder<?> inputValueCoder =
+          Preconditions.checkNotNull(outputCoders.get(inputValue));
+      // The inputValueCoder for the input PCollection should be some
+      // WindowedValueCoder of the input PCollection's element
+      // coder.
+      Preconditions.checkState(
+          inputValueCoder instanceof WindowedValue.WindowedValueCoder);
+      // The outputValueCoder for the output should be an
+      // IterableCoder of the inputValueCoder. This is a property
+      // of the backend "CollectionToSingleton" step.
+      Coder<?> outputValueCoder = IterableCoder.of(inputValueCoder);
+      addOutput(name, outputValue, outputValueCoder);
+    }
+
+    /**
+     * Adds an output with the given name to the previously added
+     * Dataflow step, producing the specified output {@code PValue}
+     * with the given {@code Coder} (if not {@code null}).
+     */
+    private void addOutput(String name, PValue value, Coder<?> valueCoder) {
+      registerOutputName(value, name);
+
+      Map<String, Object> properties = getProperties();
+      @Nullable List<Map<String, Object>> outputInfoList = null;
+      try {
+        // TODO: This should be done via a Structs accessor.
+        outputInfoList = (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
+      } catch (Exception e) {
+        throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
+      }
+      if (outputInfoList == null) {
+        outputInfoList = new ArrayList<>();
+        // TODO: This should be done via a Structs accessor.
+        properties.put(PropertyNames.OUTPUT_INFO, outputInfoList);
+      }
+
+      Map<String, Object> outputInfo = new HashMap<>();
+      addString(outputInfo, PropertyNames.OUTPUT_NAME, name);
+      addString(outputInfo, PropertyNames.USER_NAME, value.getName());
+      if (value instanceof PCollection
+          && runner.doesPCollectionRequireIndexedFormat((PCollection<?>) value)) {
+        addBoolean(outputInfo, PropertyNames.USE_INDEXED_FORMAT, true);
+      }
+      if (valueCoder != null) {
+        // Verify that encoding can be decoded, in order to catch serialization
+        // failures as early as possible.
+        CloudObject encoding = SerializableUtils.ensureSerializable(valueCoder);
+        addObject(outputInfo, PropertyNames.ENCODING, encoding);
+        outputCoders.put(value, valueCoder);
+      }
+
+      outputInfoList.add(outputInfo);
+    }
+
+    private void addDisplayData(String name, DisplayData displayData) {
+      List<Map<String, Object>> list = MAPPER.convertValue(displayData, List.class);
+      addList(getProperties(), name, list);
+    }
+
+    @Override
+    public OutputReference asOutputReference(PValue value) {
+      AppliedPTransform<?, ?, ?> transform =
+          value.getProducingTransformInternal();
+      String stepName = stepNames.get(transform);
+      if (stepName == null) {
+        throw new IllegalArgumentException(transform + " doesn't have a name specified");
+      }
+
+      String outputName = outputNames.get(value);
+      if (outputName == null) {
+        throw new IllegalArgumentException(
+            "output " + value + " doesn't have a name specified");
+      }
+
+      return new OutputReference(stepName, outputName);
+    }
+
+    private Map<String, Object> getProperties() {
+      Map<String, Object> properties = currentStep.getProperties();
+      if (properties == null) {
+        properties = new HashMap<>();
+        currentStep.setProperties(properties);
+      }
+      return properties;
+    }
+
+    /**
+     * Returns a fresh Dataflow step name.
+     */
+    private String genStepName() {
+      return "s" + (stepNames.size() + 1);
+    }
+
+    /**
+     * Records the name of the given output PValue,
+     * within its producing transform.
+     */
+    private void registerOutputName(POutput value, String name) {
+      if (outputNames.put(value, name) != null) {
+        throw new IllegalArgumentException(
+            "output " + value + " already has a name specified");
+      }
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public String toString() {
+    return "DataflowPipelineTranslator#" + hashCode();
+  }
+
+
+  ///////////////////////////////////////////////////////////////////////////
+
+  static {
+    registerTransformTranslator(
+        View.CreatePCollectionView.class,
+        new TransformTranslator<View.CreatePCollectionView>() {
+          @Override
+          public void translate(
+              View.CreatePCollectionView transform,
+              TranslationContext context) {
+            translateTyped(transform, context);
+          }
+
+          private <ElemT, ViewT> void translateTyped(
+              View.CreatePCollectionView<ElemT, ViewT> transform,
+              TranslationContext context) {
+            context.addStep(transform, "CollectionToSingleton");
+            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+            context.addCollectionToSingletonOutput(
+                PropertyNames.OUTPUT,
+                context.getInput(transform),
+                context.getOutput(transform));
+          }
+        });
+
+    DataflowPipelineTranslator.registerTransformTranslator(
+        Combine.GroupedValues.class,
+        new DataflowPipelineTranslator.TransformTranslator<Combine.GroupedValues>() {
+          @Override
+          public void translate(
+              Combine.GroupedValues transform,
+              DataflowPipelineTranslator.TranslationContext context) {
+            translateHelper(transform, context);
+          }
+
+          private <K, InputT, OutputT> void translateHelper(
+              final Combine.GroupedValues<K, InputT, OutputT> transform,
+              DataflowPipelineTranslator.TranslationContext context) {
+            context.addStep(transform, "CombineValues");
+            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+
+            AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn =
+                transform.getAppliedFn(
+                    context.getInput(transform).getPipeline().getCoderRegistry(),
+                context.getInput(transform).getCoder(),
+                context.getInput(transform).getWindowingStrategy());
+
+            context.addEncodingInput(fn.getAccumulatorCoder());
+            context.addInput(
+                PropertyNames.SERIALIZED_FN,
+                byteArrayToJsonString(serializeToByteArray(fn)));
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+          }
+        });
+
+    registerTransformTranslator(
+        Create.Values.class,
+        new TransformTranslator<Create.Values>() {
+          @Override
+          public void translate(
+              Create.Values transform,
+              TranslationContext context) {
+            createHelper(transform, context);
+          }
+
+          private <T> void createHelper(
+              Create.Values<T> transform,
+              TranslationContext context) {
+            context.addStep(transform, "CreateCollection");
+
+            Coder<T> coder = context.getOutput(transform).getCoder();
+            List<CloudObject> elements = new LinkedList<>();
+            for (T elem : transform.getElements()) {
+              byte[] encodedBytes;
+              try {
+                encodedBytes = encodeToByteArray(coder, elem);
+              } catch (CoderException exn) {
+                // TODO: Put in better element printing:
+                // truncate if too long.
+                throw new IllegalArgumentException(
+                    "Unable to encode element '" + elem + "' of transform '" + transform
+                    + "' using coder '" + coder + "'.",
+                    exn);
+              }
+              String encodedJson = byteArrayToJsonString(encodedBytes);
+              assert Arrays.equals(encodedBytes,
+                                   jsonStringToByteArray(encodedJson));
+              elements.add(CloudObject.forString(encodedJson));
+            }
+            context.addInput(PropertyNames.ELEMENT, elements);
+            context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+          }
+        });
+
+    registerTransformTranslator(
+        Flatten.FlattenPCollectionList.class,
+        new TransformTranslator<Flatten.FlattenPCollectionList>() {
+          @Override
+          public void translate(
+              Flatten.FlattenPCollectionList transform,
+              TranslationContext context) {
+            flattenHelper(transform, context);
+          }
+
+          private <T> void flattenHelper(
+              Flatten.FlattenPCollectionList<T> transform,
+              TranslationContext context) {
+            context.addStep(transform, "Flatten");
+
+            List<OutputReference> inputs = new LinkedList<>();
+            for (PCollection<T> input : context.getInput(transform).getAll()) {
+              inputs.add(context.asOutputReference(input));
+            }
+            context.addInput(PropertyNames.INPUTS, inputs);
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+          }
+        });
+
+    registerTransformTranslator(
+        GroupByKeyAndSortValuesOnly.class,
+        new TransformTranslator<GroupByKeyAndSortValuesOnly>() {
+          @Override
+          public void translate(
+              GroupByKeyAndSortValuesOnly transform,
+              TranslationContext context) {
+            groupByKeyAndSortValuesHelper(transform, context);
+          }
+
+          private <K1, K2, V> void groupByKeyAndSortValuesHelper(
+              GroupByKeyAndSortValuesOnly<K1, K2, V> transform,
+              TranslationContext context) {
+            context.addStep(transform, "GroupByKey");
+            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+            context.addInput(PropertyNames.SORT_VALUES, true);
+
+            // TODO: Add support for combiner lifting once the need arises.
+            context.addInput(
+                PropertyNames.DISALLOW_COMBINER_LIFTING, true);
+          }
+        });
+
+    registerTransformTranslator(
+        GroupByKey.class,
+        new TransformTranslator<GroupByKey>() {
+          @Override
+          public void translate(
+              GroupByKey transform,
+              TranslationContext context) {
+            groupByKeyHelper(transform, context);
+          }
+
+          private <K, V> void groupByKeyHelper(
+              GroupByKey<K, V> transform,
+              TranslationContext context) {
+            context.addStep(transform, "GroupByKey");
+            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+
+            WindowingStrategy<?, ?> windowingStrategy =
+                context.getInput(transform).getWindowingStrategy();
+            boolean isStreaming =
+                context.getPipelineOptions().as(StreamingOptions.class).isStreaming();
+            boolean disallowCombinerLifting =
+                !windowingStrategy.getWindowFn().isNonMerging()
+                || (isStreaming && !transform.fewKeys())
+                // TODO: Allow combiner lifting on the non-default trigger, as appropriate.
+                || !(windowingStrategy.getTrigger().getSpec() instanceof DefaultTrigger);
+            context.addInput(
+                PropertyNames.DISALLOW_COMBINER_LIFTING, disallowCombinerLifting);
+            context.addInput(
+                PropertyNames.SERIALIZED_FN,
+                byteArrayToJsonString(serializeToByteArray(windowingStrategy)));
+            context.addInput(
+                PropertyNames.IS_MERGING_WINDOW_FN,
+                !windowingStrategy.getWindowFn().isNonMerging());
+          }
+        });
+
+    registerTransformTranslator(
+        ParDo.BoundMulti.class,
+        new TransformTranslator<ParDo.BoundMulti>() {
+          @Override
+          public void translate(
+              ParDo.BoundMulti transform,
+              TranslationContext context) {
+            translateMultiHelper(transform, context);
+          }
+
+          private <InputT, OutputT> void translateMultiHelper(
+              ParDo.BoundMulti<InputT, OutputT> transform,
+              TranslationContext context) {
+            context.addStep(transform, "ParallelDo");
+            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+            translateFn(transform.getFn(), context.getInput(transform).getWindowingStrategy(),
+                transform.getSideInputs(), context.getInput(transform).getCoder(), context);
+            translateOutputs(context.getOutput(transform), context);
+          }
+        });
+
+    registerTransformTranslator(
+        ParDo.Bound.class,
+        new TransformTranslator<ParDo.Bound>() {
+          @Override
+          public void translate(
+              ParDo.Bound transform,
+              TranslationContext context) {
+            translateSingleHelper(transform, context);
+          }
+
+          private <InputT, OutputT> void translateSingleHelper(
+              ParDo.Bound<InputT, OutputT> transform,
+              TranslationContext context) {
+            context.addStep(transform, "ParallelDo");
+            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+            translateFn(
+                transform.getFn(),
+                context.getInput(transform).getWindowingStrategy(),
+                transform.getSideInputs(), context.getInput(transform).getCoder(), context);
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+          }
+        });
+
+    registerTransformTranslator(
+        Window.Bound.class,
+        new DataflowPipelineTranslator.TransformTranslator<Window.Bound>() {
+          @Override
+          public void translate(
+              Window.Bound transform, TranslationContext context) {
+            translateHelper(transform, context);
+          }
+
+          private <T> void translateHelper(
+              Window.Bound<T> transform, TranslationContext context) {
+            context.addStep(transform, "Bucket");
+            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+
+            WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy();
+            byte[] serializedBytes = serializeToByteArray(strategy);
+            String serializedJson = byteArrayToJsonString(serializedBytes);
+            assert Arrays.equals(serializedBytes,
+                                 jsonStringToByteArray(serializedJson));
+            context.addInput(PropertyNames.SERIALIZED_FN, serializedJson);
+          }
+        });
+
+    ///////////////////////////////////////////////////////////////////////////
+    // IO Translation.
+
+    registerTransformTranslator(
+        BigQueryIO.Read.Bound.class, new BigQueryIOTranslator.ReadTranslator());
+    registerTransformTranslator(
+        BigQueryIO.Write.Bound.class, new BigQueryIOTranslator.WriteTranslator());
+
+    registerTransformTranslator(
+        PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator());
+    registerTransformTranslator(
+        DataflowPipelineRunner.StreamingPubsubIOWrite.class,
+        new PubsubIOTranslator.WriteTranslator());
+
+    registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
+  }
+
+  private static void translateInputs(
+      PCollection<?> input,
+      List<PCollectionView<?>> sideInputs,
+      TranslationContext context) {
+    context.addInput(PropertyNames.PARALLEL_INPUT, input);
+    translateSideInputs(sideInputs, context);
+  }
+
+  // Used for ParDo
+  private static void translateSideInputs(
+      List<PCollectionView<?>> sideInputs,
+      TranslationContext context) {
+    Map<String, Object> nonParInputs = new HashMap<>();
+
+    for (PCollectionView<?> view : sideInputs) {
+      nonParInputs.put(
+          view.getTagInternal().getId(),
+          context.asOutputReference(view));
+    }
+
+    context.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs);
+  }
+
+  private static void translateFn(
+      DoFn fn,
+      WindowingStrategy windowingStrategy,
+      Iterable<PCollectionView<?>> sideInputs,
+      Coder inputCoder,
+      TranslationContext context) {
+    context.addInput(PropertyNames.USER_FN, fn.getClass().getName());
+    context.addInput(
+        PropertyNames.SERIALIZED_FN,
+        byteArrayToJsonString(serializeToByteArray(
+            new DoFnInfo(fn, windowingStrategy, sideInputs, inputCoder))));
+  }
+
+  private static void translateOutputs(
+      PCollectionTuple outputs,
+      TranslationContext context) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> entry
+             : outputs.getAll().entrySet()) {
+      TupleTag<?> tag = entry.getKey();
+      PCollection<?> output = entry.getValue();
+      context.addOutput(tag.getId(), output);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
new file mode 100644
index 0000000..8b9f3f4
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
@@ -0,0 +1,33 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import javax.annotation.Nullable;
+
+/**
+ * Signals there was an error retrieving information about a job from the Cloud Dataflow Service.
+ */
+public class DataflowServiceException extends DataflowJobException {
+  DataflowServiceException(DataflowPipelineJob job, String message) {
+    this(job, message, null);
+  }
+
+  DataflowServiceException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+    super(job, message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
new file mode 100644
index 0000000..b9d67ad
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+/**
+ * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)}
+ * {@link PTransform}.
+ *
+ * For an application of {@link Window#into(WindowFn)} that changes the {@link WindowFn}, applies
+ * a primitive {@link PTransform} in the Dataflow service.
+ *
+ * For an application of {@link Window#into(WindowFn)} that does not change the {@link WindowFn},
+ * applies an identity {@link ParDo} and sets the windowing strategy of the output
+ * {@link PCollection}.
+ *
+ * For internal use only.
+ *
+ * @param <T> the type of input element
+ */
+public class AssignWindows<T> extends PTransform<PCollection<T>, PCollection<T>> {
+  private final Window.Bound<T> transform;
+
+  /**
+   * Builds an instance of this class from the overriden transform.
+   */
+  @SuppressWarnings("unused") // Used via reflection
+  public AssignWindows(Window.Bound<T> transform) {
+    this.transform = transform;
+  }
+
+  @Override
+  public PCollection<T> apply(PCollection<T> input) {
+    WindowingStrategy<?, ?> outputStrategy =
+        transform.getOutputStrategyInternal(input.getWindowingStrategy());
+    if (transform.getWindowFn() != null) {
+      // If the windowFn changed, we create a primitive, and run the AssignWindows operation here.
+      return PCollection.<T>createPrimitiveOutputInternal(
+                            input.getPipeline(), outputStrategy, input.isBounded());
+    } else {
+      // If the windowFn didn't change, we just run a pass-through transform and then set the
+      // new windowing strategy.
+      return input.apply(ParDo.named("Identity").of(new DoFn<T, T>() {
+        @Override
+        public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
+          c.output(c.element());
+        }
+      })).setWindowingStrategyInternal(outputStrategy);
+    }
+  }
+
+  @Override
+  public void validate(PCollection<T> input) {
+    transform.validate(input);
+  }
+
+  @Override
+  protected Coder<?> getDefaultOutputCoder(PCollection<T> input) {
+    return input.getCoder();
+  }
+
+  @Override
+  protected String getKindString() {
+    return "Window.Into()";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
new file mode 100755
index 0000000..203ce4f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
@@ -0,0 +1,126 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import com.google.api.client.json.JsonFactory;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.Transport;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * BigQuery transform support code for the Dataflow backend.
+ */
+public class BigQueryIOTranslator {
+  private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
+  private static final Logger LOG = LoggerFactory.getLogger(BigQueryIOTranslator.class);
+
+  /**
+   * Implements BigQueryIO Read translation for the Dataflow backend.
+   */
+  public static class ReadTranslator
+      implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Read.Bound> {
+
+    @Override
+    public void translate(
+        BigQueryIO.Read.Bound transform, DataflowPipelineTranslator.TranslationContext context) {
+      // Actual translation.
+      context.addStep(transform, "ParallelRead");
+      context.addInput(PropertyNames.FORMAT, "bigquery");
+      context.addInput(PropertyNames.BIGQUERY_EXPORT_FORMAT, "FORMAT_AVRO");
+
+      if (transform.getQuery() != null) {
+        context.addInput(PropertyNames.BIGQUERY_QUERY, transform.getQuery());
+        context.addInput(PropertyNames.BIGQUERY_FLATTEN_RESULTS, transform.getFlattenResults());
+      } else {
+        TableReference table = transform.getTable();
+        if (table.getProjectId() == null) {
+          // If user does not specify a project we assume the table to be located in the project
+          // that owns the Dataflow job.
+          String projectIdFromOptions = context.getPipelineOptions().getProject();
+          LOG.warn(String.format(BigQueryIO.SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
+              table.getDatasetId(), table.getTableId(), projectIdFromOptions));
+          table.setProjectId(projectIdFromOptions);
+        }
+
+        context.addInput(PropertyNames.BIGQUERY_TABLE, table.getTableId());
+        context.addInput(PropertyNames.BIGQUERY_DATASET, table.getDatasetId());
+        if (table.getProjectId() != null) {
+          context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
+        }
+      }
+      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+    }
+  }
+
+  /**
+   * Implements BigQueryIO Write translation for the Dataflow backend.
+   */
+  public static class WriteTranslator
+      implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Write.Bound> {
+
+    @Override
+    public void translate(BigQueryIO.Write.Bound transform,
+                          DataflowPipelineTranslator.TranslationContext context) {
+      if (context.getPipelineOptions().isStreaming()) {
+        // Streaming is handled by the streaming runner.
+        throw new AssertionError(
+            "BigQueryIO is specified to use streaming write in batch mode.");
+      }
+
+      TableReference table = transform.getTable();
+
+      // Actual translation.
+      context.addStep(transform, "ParallelWrite");
+      context.addInput(PropertyNames.FORMAT, "bigquery");
+      context.addInput(PropertyNames.BIGQUERY_TABLE,
+                       table.getTableId());
+      context.addInput(PropertyNames.BIGQUERY_DATASET,
+                       table.getDatasetId());
+      if (table.getProjectId() != null) {
+        context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
+      }
+      if (transform.getSchema() != null) {
+        try {
+          context.addInput(PropertyNames.BIGQUERY_SCHEMA,
+                           JSON_FACTORY.toString(transform.getSchema()));
+        } catch (IOException exn) {
+          throw new IllegalArgumentException("Invalid table schema.", exn);
+        }
+      }
+      context.addInput(
+          PropertyNames.BIGQUERY_CREATE_DISPOSITION,
+          transform.getCreateDisposition().name());
+      context.addInput(
+          PropertyNames.BIGQUERY_WRITE_DISPOSITION,
+          transform.getWriteDisposition().name());
+      // Set sink encoding to TableRowJsonCoder.
+      context.addEncodingInput(
+          WindowedValue.getValueOnlyCoder(TableRowJsonCoder.of()));
+      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
new file mode 100755
index 0000000..3473c41
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
@@ -0,0 +1,119 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import static com.google.api.client.util.Base64.encodeBase64String;
+import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
+import static com.google.cloud.dataflow.sdk.util.Structs.addString;
+import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.services.dataflow.model.SourceMetadata;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Source;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.util.CloudObject;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ByteString;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * A helper class for supporting sources defined as {@code Source}.
+ *
+ * <p>Provides a bridge between the high-level {@code Source} API and the
+ * low-level {@code CloudSource} class.
+ */
+public class CustomSources {
+  private static final String SERIALIZED_SOURCE = "serialized_source";
+  @VisibleForTesting static final String SERIALIZED_SOURCE_SPLITS = "serialized_source_splits";
+  /**
+   * The current limit on the size of a ReportWorkItemStatus RPC to Google Cloud Dataflow, which
+   * includes the initial splits, is 20 MB.
+   */
+  public static final long DATAFLOW_SPLIT_RESPONSE_API_SIZE_BYTES = 20 * (1 << 20);
+
+  private static final Logger LOG = LoggerFactory.getLogger(CustomSources.class);
+
+  private static final ByteString firstSplitKey = ByteString.copyFromUtf8("0000000000000001");
+
+  public static boolean isFirstUnboundedSourceSplit(ByteString splitKey) {
+    return splitKey.equals(firstSplitKey);
+  }
+
+  private static int getDesiredNumUnboundedSourceSplits(DataflowPipelineOptions options) {
+    if (options.getMaxNumWorkers() > 0) {
+      return options.getMaxNumWorkers();
+    } else if (options.getNumWorkers() > 0) {
+      return options.getNumWorkers() * 3;
+    } else {
+      return 20;
+    }
+  }
+
+  public static com.google.api.services.dataflow.model.Source serializeToCloudSource(
+      Source<?> source, PipelineOptions options) throws Exception {
+    com.google.api.services.dataflow.model.Source cloudSource =
+        new com.google.api.services.dataflow.model.Source();
+    // We ourselves act as the SourceFormat.
+    cloudSource.setSpec(CloudObject.forClass(CustomSources.class));
+    addString(
+        cloudSource.getSpec(), SERIALIZED_SOURCE, encodeBase64String(serializeToByteArray(source)));
+
+    SourceMetadata metadata = new SourceMetadata();
+    if (source instanceof BoundedSource) {
+      BoundedSource<?> boundedSource = (BoundedSource<?>) source;
+      try {
+        metadata.setProducesSortedKeys(boundedSource.producesSortedKeys(options));
+      } catch (Exception e) {
+        LOG.warn("Failed to check if the source produces sorted keys: " + source, e);
+      }
+
+      // Size estimation is best effort so we continue even if it fails here.
+      try {
+        metadata.setEstimatedSizeBytes(boundedSource.getEstimatedSizeBytes(options));
+      } catch (Exception e) {
+        LOG.warn("Size estimation of the source failed: " + source, e);
+      }
+    } else if (source instanceof UnboundedSource) {
+      UnboundedSource<?, ?> unboundedSource = (UnboundedSource<?, ?>) source;
+      metadata.setInfinite(true);
+      List<String> encodedSplits = new ArrayList<>();
+      int desiredNumSplits =
+          getDesiredNumUnboundedSourceSplits(options.as(DataflowPipelineOptions.class));
+      for (UnboundedSource<?, ?> split :
+          unboundedSource.generateInitialSplits(desiredNumSplits, options)) {
+        encodedSplits.add(encodeBase64String(serializeToByteArray(split)));
+      }
+      checkArgument(!encodedSplits.isEmpty(), "UnboundedSources must have at least one split");
+      addStringList(cloudSource.getSpec(), SERIALIZED_SOURCE_SPLITS, encodedSplits);
+    } else {
+      throw new IllegalArgumentException("Unexpected source kind: " + source.getClass());
+    }
+
+    cloudSource.setMetadata(metadata);
+    return cloudSource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
new file mode 100755
index 0000000..5c1dd95
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
@@ -0,0 +1,80 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used.
+ */
+public class DataflowAggregatorTransforms {
+  private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms;
+  private final Multimap<PTransform<?, ?>, AppliedPTransform<?, ?, ?>> transformAppliedTransforms;
+  private final BiMap<AppliedPTransform<?, ?, ?>, String> appliedStepNames;
+
+  public DataflowAggregatorTransforms(
+      Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms,
+      Map<AppliedPTransform<?, ?, ?>, String> transformStepNames) {
+    this.aggregatorTransforms = aggregatorTransforms;
+    appliedStepNames = HashBiMap.create(transformStepNames);
+
+    transformAppliedTransforms = HashMultimap.create();
+    for (AppliedPTransform<?, ?, ?> appliedTransform : transformStepNames.keySet()) {
+      transformAppliedTransforms.put(appliedTransform.getTransform(), appliedTransform);
+    }
+  }
+
+  /**
+   * Returns true if the provided {@link Aggregator} is used in the constructing {@link Pipeline}.
+   */
+  public boolean contains(Aggregator<?, ?> aggregator) {
+    return aggregatorTransforms.containsKey(aggregator);
+  }
+
+  /**
+   * Gets the step names in which the {@link Aggregator} is used.
+   */
+  public Collection<String> getAggregatorStepNames(Aggregator<?, ?> aggregator) {
+    Collection<String> names = new HashSet<>();
+    Collection<PTransform<?, ?>> transforms = aggregatorTransforms.get(aggregator);
+    for (PTransform<?, ?> transform : transforms) {
+      for (AppliedPTransform<?, ?, ?> applied : transformAppliedTransforms.get(transform)) {
+        names.add(appliedStepNames.get(applied));
+      }
+    }
+    return names;
+  }
+
+  /**
+   * Gets the {@link PTransform} that was assigned the provided step name.
+   */
+  public AppliedPTransform<?, ?, ?> getAppliedTransformForStepName(String stepName) {
+    return appliedStepNames.inverse().get(stepName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
new file mode 100755
index 0000000..5fc1eb7
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Methods for extracting the values of an {@link Aggregator} from a collection of {@link
+ * MetricUpdate MetricUpdates}.
+ */
+public final class DataflowMetricUpdateExtractor {
+  private static final String STEP_NAME_CONTEXT_KEY = "step";
+  private static final String IS_TENTATIVE_KEY = "tentative";
+
+  private DataflowMetricUpdateExtractor() {
+    // Do not instantiate.
+  }
+
+  /**
+   * Extract the values of the provided {@link Aggregator} at each {@link PTransform} it was used in
+   * according to the provided {@link DataflowAggregatorTransforms} from the given list of {@link
+   * MetricUpdate MetricUpdates}.
+   */
+  public static <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator,
+      DataflowAggregatorTransforms aggregatorTransforms, List<MetricUpdate> metricUpdates) {
+    Map<String, OutputT> results = new HashMap<>();
+    if (metricUpdates == null) {
+      return results;
+    }
+
+    String aggregatorName = aggregator.getName();
+    Collection<String> aggregatorSteps = aggregatorTransforms.getAggregatorStepNames(aggregator);
+
+    for (MetricUpdate metricUpdate : metricUpdates) {
+      MetricStructuredName metricStructuredName = metricUpdate.getName();
+      Map<String, String> context = metricStructuredName.getContext();
+      if (metricStructuredName.getName().equals(aggregatorName) && context != null
+          && aggregatorSteps.contains(context.get(STEP_NAME_CONTEXT_KEY))) {
+        AppliedPTransform<?, ?, ?> transform =
+            aggregatorTransforms.getAppliedTransformForStepName(
+                context.get(STEP_NAME_CONTEXT_KEY));
+        String fullName = transform.getFullName();
+        // Prefer the tentative (fresher) value if it exists.
+        if (Boolean.parseBoolean(context.get(IS_TENTATIVE_KEY)) || !results.containsKey(fullName)) {
+          results.put(fullName, toValue(aggregator, metricUpdate));
+        }
+      }
+    }
+
+    return results;
+
+  }
+
+  private static <OutputT> OutputT toValue(
+      Aggregator<?, OutputT> aggregator, MetricUpdate metricUpdate) {
+    CombineFn<?, ?, OutputT> combineFn = aggregator.getCombineFn();
+    Class<? super OutputT> outputType = combineFn.getOutputType().getRawType();
+
+    if (outputType.equals(Long.class)) {
+      @SuppressWarnings("unchecked")
+      OutputT asLong = (OutputT) Long.valueOf(toNumber(metricUpdate).longValue());
+      return asLong;
+    }
+    if (outputType.equals(Integer.class)) {
+      @SuppressWarnings("unchecked")
+      OutputT asInt = (OutputT) Integer.valueOf(toNumber(metricUpdate).intValue());
+      return asInt;
+    }
+    if (outputType.equals(Double.class)) {
+      @SuppressWarnings("unchecked")
+      OutputT asDouble = (OutputT) Double.valueOf(toNumber(metricUpdate).doubleValue());
+      return asDouble;
+    }
+    throw new UnsupportedOperationException(
+        "Unsupported Output Type " + outputType + " in aggregator " + aggregator);
+  }
+
+  private static Number toNumber(MetricUpdate update) {
+    if (update.getScalar() instanceof Number) {
+      return (Number) update.getScalar();
+    }
+    throw new IllegalArgumentException(
+        "Metric Update " + update + " does not have a numeric scalar");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
new file mode 100755
index 0000000..b54d5c6
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
@@ -0,0 +1,108 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+
+/**
+ * Pubsub transform support code for the Dataflow backend.
+ */
+public class PubsubIOTranslator {
+
+  /**
+   * Implements PubsubIO Read translation for the Dataflow backend.
+   */
+  public static class ReadTranslator<T> implements TransformTranslator<PubsubIO.Read.Bound<T>> {
+    @Override
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void translate(
+        PubsubIO.Read.Bound transform,
+        TranslationContext context) {
+      translateReadHelper(transform, context);
+    }
+
+    private <T> void translateReadHelper(
+        PubsubIO.Read.Bound<T> transform,
+        TranslationContext context) {
+      if (!context.getPipelineOptions().isStreaming()) {
+        throw new IllegalArgumentException(
+            "PubsubIO.Read can only be used with the Dataflow streaming runner.");
+      }
+
+      context.addStep(transform, "ParallelRead");
+      context.addInput(PropertyNames.FORMAT, "pubsub");
+      if (transform.getTopic() != null) {
+        context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
+      }
+      if (transform.getSubscription() != null) {
+        context.addInput(
+            PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription().asV1Beta1Path());
+      }
+      if (transform.getTimestampLabel() != null) {
+        context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
+      }
+      if (transform.getIdLabel() != null) {
+        context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
+      }
+      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+    }
+  }
+
+  /**
+   * Implements PubsubIO Write translation for the Dataflow backend.
+   */
+  public static class WriteTranslator<T>
+      implements TransformTranslator<DataflowPipelineRunner.StreamingPubsubIOWrite<T>> {
+
+    @Override
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void translate(
+        DataflowPipelineRunner.StreamingPubsubIOWrite transform,
+        TranslationContext context) {
+      translateWriteHelper(transform, context);
+    }
+
+    private <T> void translateWriteHelper(
+        DataflowPipelineRunner.StreamingPubsubIOWrite<T> customTransform,
+        TranslationContext context) {
+      if (!context.getPipelineOptions().isStreaming()) {
+        throw new IllegalArgumentException(
+            "PubsubIO.Write is non-primitive for the Dataflow batch runner.");
+      }
+
+      PubsubIO.Write.Bound<T> transform = customTransform.getOverriddenTransform();
+
+      context.addStep(customTransform, "ParallelWrite");
+      context.addInput(PropertyNames.FORMAT, "pubsub");
+      context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
+      if (transform.getTimestampLabel() != null) {
+        context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
+      }
+      if (transform.getIdLabel() != null) {
+        context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
+      }
+      context.addEncodingInput(WindowedValue.getValueOnlyCoder(transform.getCoder()));
+      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(customTransform));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
new file mode 100755
index 0000000..4998af3
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
@@ -0,0 +1,104 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners.dataflow;
+
+import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
+import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
+import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
+
+import com.google.api.services.dataflow.model.SourceMetadata;
+import com.google.cloud.dataflow.sdk.io.FileBasedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.Source;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.values.PValue;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end.
+ */
+public class ReadTranslator implements TransformTranslator<Read.Bounded<?>> {
+  @Override
+  public void translate(Read.Bounded<?> transform, TranslationContext context) {
+    translateReadHelper(transform.getSource(), transform, context);
+  }
+
+  public static <T> void translateReadHelper(Source<T> source,
+      PTransform<?, ? extends PValue> transform,
+      DataflowPipelineTranslator.TranslationContext context) {
+    try {
+      // TODO: Move this validation out of translation once IOChannelUtils is portable
+      // and can be reconstructed on the worker.
+      if (source instanceof FileBasedSource) {
+        String filePatternOrSpec = ((FileBasedSource<?>) source).getFileOrPatternSpec();
+        context.getPipelineOptions()
+               .getPathValidator()
+               .validateInputFilePatternSupported(filePatternOrSpec);
+      }
+
+      context.addStep(transform, "ParallelRead");
+      context.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT);
+      context.addInput(
+          PropertyNames.SOURCE_STEP_INPUT,
+          cloudSourceToDictionary(
+              CustomSources.serializeToCloudSource(source, context.getPipelineOptions())));
+      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  // Represents a cloud Source as a dictionary for encoding inside the {@code SOURCE_STEP_INPUT}
+  // property of CloudWorkflowStep.input.
+  private static Map<String, Object> cloudSourceToDictionary(
+      com.google.api.services.dataflow.model.Source source) {
+    // Do not translate encoding - the source's encoding is translated elsewhere
+    // to the step's output info.
+    Map<String, Object> res = new HashMap<>();
+    addDictionary(res, PropertyNames.SOURCE_SPEC, source.getSpec());
+    if (source.getMetadata() != null) {
+      addDictionary(res, PropertyNames.SOURCE_METADATA,
+          cloudSourceMetadataToDictionary(source.getMetadata()));
+    }
+    if (source.getDoesNotNeedSplitting() != null) {
+      addBoolean(
+          res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting());
+    }
+    return res;
+  }
+
+  private static Map<String, Object> cloudSourceMetadataToDictionary(SourceMetadata metadata) {
+    Map<String, Object> res = new HashMap<>();
+    if (metadata.getProducesSortedKeys() != null) {
+      addBoolean(res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys());
+    }
+    if (metadata.getEstimatedSizeBytes() != null) {
+      addLong(res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes());
+    }
+    if (metadata.getInfinite() != null) {
+      addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite());
+    }
+    return res;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
new file mode 100755
index 0000000..d1484ce
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Implementation of the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
+ */
+package com.google.cloud.dataflow.sdk.runners.dataflow;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
new file mode 100644
index 0000000..2f920a6
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
@@ -0,0 +1,27 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.testing;
+
+import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
+
+/**
+ * A set of options used to configure the {@link TestPipeline}.
+ */
+public interface TestDataflowPipelineOptions extends BlockingDataflowPipelineOptions {
+
+}


[03/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java
deleted file mode 100644
index dc822ea..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java
+++ /dev/null
@@ -1,948 +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 com.google.cloud.dataflow.sdk.runners.worker;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-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 com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.ListCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.RandomAccessData;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.MoreObjects.ToStringHelper;
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * An Ism file is a prefix encoded composite key value file broken into shards. Each composite
- * key is composed of a fixed number of component keys. A fixed number of those sub keys represent
- * the shard key portion; see {@link IsmRecord} and {@link IsmRecordCoder} for further details
- * around the data format. In addition to the data, there is a bloom filter,
- * and multiple indices to allow for efficient retrieval.
- *
- * <p>An Ism file is composed of these high level sections (in order):
- * <ul>
- *   <li>shard block</li>
- *   <li>bloom filter (See {@code ScalableBloomFilter} for details on encoding format)</li>
- *   <li>shard index</li>
- *   <li>footer (See {@link Footer} for details on encoding format)</li>
- * </ul>
- *
- * <p>The shard block is composed of multiple copies of the following:
- * <ul>
- *   <li>data block</li>
- *   <li>data index</li>
- * </ul>
- *
- * <p>The data block is composed of multiple copies of the following:
- * <ul>
- *   <li>key prefix (See {@link KeyPrefix} for details on encoding format)</li>
- *   <li>unshared key bytes</li>
- *   <li>value bytes</li>
- *   <li>optional 0x00 0x00 bytes followed by metadata bytes
- *       (if the following 0x00 0x00 bytes are not present, then there are no metadata bytes)</li>
- * </ul>
- * Each key written into the data block must be in unsigned lexicographically increasing order
- * and also its shard portion of the key must hash to the same shard id as all other keys
- * within the same data block. The hashing function used is the
- * <a href="http://smhasher.googlecode.com/svn/trunk/MurmurHash3.cpp">
- * 32-bit murmur3 algorithm, x86 variant</a> (little-endian variant),
- * using {@code 1225801234} as the seed value.
- *
- * <p>The data index is composed of {@code N} copies of the following:
- * <ul>
- *   <li>key prefix (See {@link KeyPrefix} for details on encoding format)</li>
- *   <li>unshared key bytes</li>
- *   <li>byte offset to key prefix in data block (variable length long coding)</li>
- * </ul>
- *
- * <p>The shard index is composed of a {@link VarInt variable length integer} encoding representing
- * the number of shard index records followed by that many shard index records.
- * See {@link IsmShardCoder} for further details as to its encoding scheme.
- */
-public class IsmFormat {
-  private static final int HASH_SEED = 1225801234;
-  private static final HashFunction HASH_FUNCTION = Hashing.murmur3_32(HASH_SEED);
-  static final int SHARD_BITS = 0x7F; // [0-127] shards + [128-255] metadata shards
-
-  /**
-   * A record containing a composite key and either a value or metadata. The composite key
-   * must not contain the metadata key component place holder if producing a value record, and must
-   * contain the metadata component key place holder if producing a metadata record.
-   *
-   * <p>The composite key is a fixed number of component keys where the first {@code N} component
-   * keys are used to create a shard id via hashing. See {@link IsmRecordCoder#hash(List)} for
-   * further details.
-   */
-  public static class IsmRecord<V> {
-    /** Returns an IsmRecord with the specified key components and value. */
-    public static <V> IsmRecord<V> of(List<?> keyComponents, V value) {
-      checkNotNull(keyComponents);
-      checkArgument(!keyComponents.isEmpty(), "Expected non-empty list of key components.");
-      checkArgument(!isMetadataKey(keyComponents),
-          "Expected key components to not contain metadata key.");
-      return new IsmRecord<>(keyComponents, value, null);
-    }
-
-    public static <V> IsmRecord<V> meta(List<?> keyComponents, byte[] metadata) {
-      checkNotNull(keyComponents);
-      checkNotNull(metadata);
-      checkArgument(!keyComponents.isEmpty(), "Expected non-empty list of key components.");
-      checkArgument(isMetadataKey(keyComponents),
-          "Expected key components to contain metadata key.");
-      return new IsmRecord<V>(keyComponents, null, metadata);
-    }
-
-    private final List<?> keyComponents;
-    @Nullable
-    private final V value;
-    @Nullable
-    private final byte[] metadata;
-    private IsmRecord(List<?> keyComponents, V value, byte[] metadata) {
-      this.keyComponents = keyComponents;
-      this.value = value;
-      this.metadata = metadata;
-    }
-
-    /** Returns the list of key components. */
-    public List<?> getKeyComponents() {
-      return keyComponents;
-    }
-
-    /** Returns the key component at the specified index. */
-    public Object getKeyComponent(int index) {
-      return keyComponents.get(index);
-    }
-
-    /**
-     * Returns the value. Throws {@link IllegalStateException} if this is not a
-     * value record.
-     */
-    public V getValue() {
-      checkState(!isMetadataKey(keyComponents),
-          "This is a metadata record and not a value record.");
-      return value;
-    }
-
-    /**
-     * Returns the metadata. Throws {@link IllegalStateException} if this is not a
-     * metadata record.
-     */
-    public byte[] getMetadata() {
-      checkState(isMetadataKey(keyComponents),
-          "This is a value record and not a metadata record.");
-      return metadata;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (!(obj instanceof IsmRecord)) {
-        return false;
-      }
-      IsmRecord<?> other = (IsmRecord<?>) obj;
-      return Objects.equal(keyComponents, other.keyComponents)
-          && Objects.equal(value, other.value)
-          && Arrays.equals(metadata, other.metadata);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(keyComponents, value, Arrays.hashCode(metadata));
-    }
-
-    @Override
-    public String toString() {
-      ToStringHelper builder = MoreObjects.toStringHelper(IsmRecord.class)
-          .add("keyComponents", keyComponents);
-      if (isMetadataKey(keyComponents)) {
-        builder.add("metadata", metadata);
-      } else {
-        builder.add("value", value);
-      }
-      return builder.toString();
-    }
-  }
-
-  /** A {@link Coder} for {@link IsmRecord}s.
-   *
-   * <p>Note that this coder standalone will not produce an Ism file. This coder can be used
-   * to materialize a {@link PCollection} of {@link IsmRecord}s. Only when this coder
-   * is combined with an {@link IsmSink} will one produce an Ism file.
-   *
-   * <p>The {@link IsmRecord} encoded format is:
-   * <ul>
-   *   <li>encoded key component 1 using key component coder 1</li>
-   *   <li>...</li>
-   *   <li>encoded key component N using key component coder N</li>
-   *   <li>encoded value using value coder</li>
-   * </ul>
-   */
-  public static class IsmRecordCoder<V>
-      extends StandardCoder<IsmRecord<V>> {
-    /** Returns an IsmRecordCoder with the specified key component coders, value coder. */
-    public static <V> IsmRecordCoder<V> of(
-        int numberOfShardKeyCoders,
-        int numberOfMetadataShardKeyCoders,
-        List<Coder<?>> keyComponentCoders,
-        Coder<V> valueCoder) {
-      checkNotNull(keyComponentCoders);
-      checkArgument(keyComponentCoders.size() > 0);
-      checkArgument(numberOfShardKeyCoders > 0);
-      checkArgument(numberOfShardKeyCoders <= keyComponentCoders.size());
-      checkArgument(numberOfMetadataShardKeyCoders <= keyComponentCoders.size());
-      return new IsmRecordCoder<>(
-          numberOfShardKeyCoders,
-          numberOfMetadataShardKeyCoders,
-          keyComponentCoders,
-          valueCoder);
-    }
-
-    /**
-     * Returns an IsmRecordCoder with the specified coders. Note that this method is not meant
-     * to be called by users but used by Jackson when decoding this coder.
-     */
-    @JsonCreator
-    public static IsmRecordCoder<?> of(
-        @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,
-          "Expecting at least 2 components, got " + components.size());
-      return of(
-          numberOfShardCoders,
-          numberOfMetadataShardCoders,
-          components.subList(0, components.size() - 1),
-          components.get(components.size() - 1));
-    }
-
-    private final int numberOfShardKeyCoders;
-    private final int numberOfMetadataShardKeyCoders;
-    private final List<Coder<?>> keyComponentCoders;
-    private final Coder<V> valueCoder;
-
-    private IsmRecordCoder(
-        int numberOfShardKeyCoders,
-        int numberOfMetadataShardKeyCoders,
-        List<Coder<?>> keyComponentCoders, Coder<V> valueCoder) {
-      this.numberOfShardKeyCoders = numberOfShardKeyCoders;
-      this.numberOfMetadataShardKeyCoders = numberOfMetadataShardKeyCoders;
-      this.keyComponentCoders = keyComponentCoders;
-      this.valueCoder = valueCoder;
-    }
-
-    /** Returns the list of key component coders. */
-    public List<Coder<?>> getKeyComponentCoders() {
-      return keyComponentCoders;
-    }
-
-    /** Returns the key coder at the specified index. */
-    public Coder getKeyComponentCoder(int index) {
-      return keyComponentCoders.get(index);
-    }
-
-    /** Returns the value coder. */
-    public Coder<V> getValueCoder() {
-      return valueCoder;
-    }
-
-    @Override
-    public void encode(IsmRecord<V> value, OutputStream outStream,
-        Coder.Context context) throws CoderException, IOException {
-      if (value.getKeyComponents().size() != keyComponentCoders.size()) {
-        throw new CoderException(String.format(
-            "Expected %s key component(s) but received key component(s) %s.",
-            keyComponentCoders.size(), value.getKeyComponents()));
-      }
-      for (int i = 0; i < keyComponentCoders.size(); ++i) {
-        getKeyComponentCoder(i).encode(value.getKeyComponent(i), outStream, context.nested());
-      }
-      if (isMetadataKey(value.getKeyComponents())) {
-        ByteArrayCoder.of().encode(value.getMetadata(), outStream, context.nested());
-      } else {
-        valueCoder.encode(value.getValue(), outStream, context.nested());
-      }
-    }
-
-    @Override
-    public IsmRecord<V> decode(InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      List<Object> keyComponents = new ArrayList<>(keyComponentCoders.size());
-      for (Coder<?> keyCoder : keyComponentCoders) {
-        keyComponents.add(keyCoder.decode(inStream, context.nested()));
-      }
-      if (isMetadataKey(keyComponents)) {
-        return IsmRecord.<V>meta(
-            keyComponents, ByteArrayCoder.of().decode(inStream, context.nested()));
-      } else {
-        return IsmRecord.<V>of(keyComponents, valueCoder.decode(inStream, context.nested()));
-      }
-    }
-
-    int getNumberOfShardKeyCoders(List<?> keyComponents) {
-      if (isMetadataKey(keyComponents)) {
-        return numberOfMetadataShardKeyCoders;
-      } else {
-        return numberOfShardKeyCoders;
-      }
-    }
-
-    /**
-     * Computes the shard id for the given key component(s).
-     *
-     * The shard keys are encoded into their byte representations and hashed using the
-     * <a href="http://smhasher.googlecode.com/svn/trunk/MurmurHash3.cpp">
-     * 32-bit murmur3 algorithm, x86 variant</a> (little-endian variant),
-     * using {@code 1225801234} as the seed value. We ensure that shard ids for
-     * metadata keys and normal keys do not overlap.
-     */
-    public <V, T> int hash(List<?> keyComponents) {
-      return encodeAndHash(keyComponents, new RandomAccessData(), new ArrayList<Integer>());
-    }
-
-    /**
-     * Computes the shard id for the given key component(s).
-     *
-     * Mutates {@code keyBytes} such that when returned, contains the encoded
-     * version of the key components.
-     */
-    <V, T> int encodeAndHash(List<?> keyComponents, RandomAccessData keyBytesToMutate) {
-      return encodeAndHash(keyComponents, keyBytesToMutate, new ArrayList<Integer>());
-    }
-
-    /**
-     * Computes the shard id for the given key component(s).
-     *
-     * Mutates {@code keyBytes} such that when returned, contains the encoded
-     * version of the key components. Also, mutates {@code keyComponentByteOffsetsToMutate} to
-     * store the location where each key component's encoded byte representation ends within
-     * {@code keyBytes}.
-     */
-    <V, T> int encodeAndHash(
-        List<?> keyComponents,
-        RandomAccessData keyBytesToMutate,
-        List<Integer> keyComponentByteOffsetsToMutate) {
-      checkNotNull(keyComponents);
-      checkArgument(keyComponents.size() <= keyComponentCoders.size(),
-          "Expected at most %s key component(s) but received %s.",
-          keyComponentCoders.size(), keyComponents);
-
-      final int numberOfKeyCodersToUse;
-      final int shardOffset;
-      if (isMetadataKey(keyComponents)) {
-        numberOfKeyCodersToUse = numberOfMetadataShardKeyCoders;
-        shardOffset = SHARD_BITS + 1;
-      } else {
-        numberOfKeyCodersToUse = numberOfShardKeyCoders;
-        shardOffset = 0;
-      }
-
-      checkArgument(numberOfKeyCodersToUse <= keyComponents.size(),
-          "Expected at least %s key component(s) but received %s.",
-          numberOfShardKeyCoders, keyComponents);
-
-      try {
-        // Encode the shard portion
-        for (int i = 0; i < numberOfKeyCodersToUse; ++i) {
-          getKeyComponentCoder(i).encode(
-              keyComponents.get(i), keyBytesToMutate.asOutputStream(), Context.NESTED);
-          keyComponentByteOffsetsToMutate.add(keyBytesToMutate.size());
-        }
-        int rval = HASH_FUNCTION.hashBytes(
-            keyBytesToMutate.array(), 0, keyBytesToMutate.size()).asInt() & SHARD_BITS;
-        rval += shardOffset;
-
-        // Encode the remainder
-        for (int i = numberOfKeyCodersToUse; i < keyComponents.size(); ++i) {
-          getKeyComponentCoder(i).encode(
-              keyComponents.get(i), keyBytesToMutate.asOutputStream(), Context.NESTED);
-          keyComponentByteOffsetsToMutate.add(keyBytesToMutate.size());
-        }
-        return rval;
-      } catch (IOException e) {
-        throw new IllegalStateException(
-            String.format("Failed to hash %s with coder %s", keyComponents, this), e);
-      }
-    }
-
-    @Override
-    public List<Coder<?>> getCoderArguments() {
-      return ImmutableList.<Coder<?>>builder()
-          .addAll(keyComponentCoders)
-          .add(valueCoder)
-          .build();
-    }
-
-    @Override
-    public CloudObject asCloudObject() {
-      CloudObject cloudObject = super.asCloudObject();
-      addLong(cloudObject, PropertyNames.NUM_SHARD_CODERS, numberOfShardKeyCoders);
-      addLong(cloudObject, PropertyNames.NUM_METADATA_SHARD_CODERS, numberOfMetadataShardKeyCoders);
-      return cloudObject;
-    }
-
-    @Override
-    public void verifyDeterministic() throws Coder.NonDeterministicException {
-      verifyDeterministic("Key component coders expected to be deterministic.", keyComponentCoders);
-      verifyDeterministic("Value coder expected to be deterministic.", valueCoder);
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      for (Coder<?> keyComponentCoder : keyComponentCoders) {
-        if (!keyComponentCoder.consistentWithEquals()) {
-          return false;
-        }
-      }
-      return valueCoder.consistentWithEquals();
-    }
-
-    @Override
-    public Object structuralValue(IsmRecord<V> record) throws Exception {
-      checkState(record.getKeyComponents().size() == keyComponentCoders.size(),
-          "Expected the number of key component coders %s "
-          + "to match the number of key components %s.",
-          keyComponentCoders.size(), record.getKeyComponents());
-
-      if (record != null && consistentWithEquals()) {
-        ArrayList<Object> keyComponentStructuralValues = new ArrayList<>();
-        for (int i = 0; i < keyComponentCoders.size(); ++i) {
-          keyComponentStructuralValues.add(
-              getKeyComponentCoder(i).structuralValue(record.getKeyComponent(i)));
-        }
-        if (isMetadataKey(record.getKeyComponents())) {
-          return IsmRecord.meta(keyComponentStructuralValues, record.getMetadata());
-        } else {
-          return IsmRecord.of(keyComponentStructuralValues,
-              valueCoder.structuralValue(record.getValue()));
-        }
-      }
-      return super.structuralValue(record);
-    }
-  }
-
-  /**
-   * Validates that the key portion of the given coder is deterministic.
-   */
-  static void validateCoderIsCompatible(IsmRecordCoder<?> coder) {
-    for (Coder<?> keyComponentCoder : coder.getKeyComponentCoders()) {
-      try {
-          keyComponentCoder.verifyDeterministic();
-      } catch (NonDeterministicException e) {
-        throw new IllegalArgumentException(
-            String.format("Key component coder %s is expected to be deterministic.",
-                keyComponentCoder), e);
-      }
-    }
-  }
-
-  /** Returns true if and only if any of the passed in key components represent a metadata key. */
-  public static boolean isMetadataKey(List<?> keyComponents) {
-    for (Object keyComponent : keyComponents) {
-      if (keyComponent == METADATA_KEY) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /** A marker object representing the wildcard metadata key component. */
-  private static final Object METADATA_KEY = new Object() {
-    @Override
-    public String toString() {
-      return "META";
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return this == obj;
-    }
-
-    @Override
-    public int hashCode() {
-      return -1248902349;
-    }
-  };
-
-  /**
-   * An object representing a wild card for a key component.
-   * Encoded using {@link MetadataKeyCoder}.
-   */
-  public static Object getMetadataKey() {
-    return METADATA_KEY;
-  }
-
-  /**
-   * A coder for metadata key component. Can be used to wrap key component coder allowing for
-   * the metadata key component to be used as a place holder instead of an actual key.
-   */
-  public static class MetadataKeyCoder<K> extends StandardCoder<K> {
-    public static <K> MetadataKeyCoder<K> of(Coder<K> keyCoder) {
-      checkNotNull(keyCoder);
-      return new MetadataKeyCoder<>(keyCoder);
-    }
-
-    /**
-     * Returns an IsmRecordCoder with the specified coders. Note that this method is not meant
-     * to be called by users but used by Jackson when decoding this coder.
-     */
-    @JsonCreator
-    public static MetadataKeyCoder<?> of(
-        @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
-      Preconditions.checkArgument(components.size() == 1,
-          "Expecting one component, got " + components.size());
-      return of(components.get(0));
-    }
-
-    private final Coder<K> keyCoder;
-
-    private MetadataKeyCoder(Coder<K> keyCoder) {
-      this.keyCoder = keyCoder;
-    }
-
-    public Coder<K> getKeyCoder() {
-      return keyCoder;
-    }
-
-    @Override
-    public void encode(K value, OutputStream outStream, Coder.Context context)
-        throws CoderException, IOException {
-      if (value == METADATA_KEY) {
-        outStream.write(0);
-      } else {
-        outStream.write(1);
-        keyCoder.encode(value, outStream, context.nested());
-      }
-    }
-
-    @Override
-    public K decode(InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      int marker = inStream.read();
-      if (marker == 0) {
-        return (K) getMetadataKey();
-      } else if (marker == 1) {
-        return keyCoder.decode(inStream, context.nested());
-      } else {
-        throw new CoderException(String.format("Expected marker but got %s.", marker));
-      }
-    }
-
-    @Override
-    public List<Coder<?>> getCoderArguments() {
-      return ImmutableList.<Coder<?>>of(keyCoder);
-    }
-
-    @Override
-    public void verifyDeterministic() throws NonDeterministicException {
-      verifyDeterministic("Expected key coder to be deterministic", keyCoder);
-    }
-  }
-
-  /**
-   * A shard descriptor containing shard id, the data block offset, and the index offset for the
-   * given shard.
-   */
-  public static class IsmShard {
-    private final int id;
-    private final long blockOffset;
-    private final long indexOffset;
-
-    /** Returns an IsmShard with the given id, block offset and no index offset. */
-    public static IsmShard of(int id, long blockOffset) {
-      IsmShard ismShard = new IsmShard(id, blockOffset, -1);
-      checkState(id >= 0,
-          "%s attempting to be written with negative shard id.",
-          ismShard);
-      checkState(blockOffset >= 0,
-          "%s attempting to be written with negative block offset.",
-          ismShard);
-      return ismShard;
-    }
-
-    /** Returns an IsmShard with the given id, block offset, and index offset. */
-    public static IsmShard of(int id, long blockOffset, long indexOffset) {
-      IsmShard ismShard = new IsmShard(id, blockOffset, indexOffset);
-      checkState(id >= 0,
-          "%s attempting to be written with negative shard id.",
-          ismShard);
-      checkState(blockOffset >= 0,
-          "%s attempting to be written with negative block offset.",
-          ismShard);
-      checkState(indexOffset >= 0,
-          "%s attempting to be written with negative index offset.",
-          ismShard);
-      return ismShard;
-    }
-
-    private IsmShard(int id, long blockOffset, long indexOffset) {
-      this.id = id;
-      this.blockOffset = blockOffset;
-      this.indexOffset = indexOffset;
-    }
-
-    /** Return the shard id. */
-    public int getId() {
-      return id;
-    }
-
-    /** Return the absolute position within the Ism file where the data block begins. */
-    public long getBlockOffset() {
-      return blockOffset;
-    }
-
-    /**
-     * Return the absolute position within the Ism file where the index block begins.
-     * Throws {@link IllegalStateException} if the index offset was never specified.
-     */
-    public long getIndexOffset() {
-      checkState(indexOffset >= 0,
-            "Unable to fetch index offset because it was never specified.");
-      return indexOffset;
-    }
-
-    /** Returns a new IsmShard like this one with the specified index offset. */
-    public IsmShard withIndexOffset(long indexOffset) {
-      return of(id, blockOffset, indexOffset);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(IsmShard.class)
-          .add("id", id)
-          .add("blockOffset", blockOffset)
-          .add("indexOffset", indexOffset)
-          .toString();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (!(obj instanceof IsmShard)) {
-        return false;
-      }
-      IsmShard other = (IsmShard) obj;
-      return Objects.equal(id, other.id)
-          && Objects.equal(blockOffset, other.blockOffset)
-          && Objects.equal(indexOffset, other.indexOffset);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(id, blockOffset, indexOffset);
-    }
-  }
-
-  /**
-   * A {@link ListCoder} wrapping a {@link IsmShardCoder} used to encode the shard index.
-   * See {@link ListCoder} for its encoding specification and {@link IsmShardCoder} for its
-   * encoding specification.
-   */
-  public static final Coder<List<IsmShard>> ISM_SHARD_INDEX_CODER =
-      ListCoder.of(IsmShardCoder.of());
-
-  /**
-   * A coder for {@link IsmShard}s.
-   *
-   * The shard descriptor is encoded as:
-   * <ul>
-   *   <li>id (variable length integer encoding)</li>
-   *   <li>blockOffset (variable length long encoding)</li>
-   *   <li>indexOffset (variable length long encoding)</li>
-   * </ul>
-   */
-  public static class IsmShardCoder extends AtomicCoder<IsmShard> {
-    private static final IsmShardCoder INSTANCE = new IsmShardCoder();
-
-    /** Returns an IsmShardCoder. */
-    @JsonCreator
-    public static IsmShardCoder of() {
-      return INSTANCE;
-    }
-
-    private IsmShardCoder() {
-    }
-
-    @Override
-    public void encode(IsmShard value, OutputStream outStream, Coder.Context context)
-        throws CoderException, IOException {
-      checkState(value.getIndexOffset() >= 0,
-          "%s attempting to be written without index offset.",
-          value);
-      VarIntCoder.of().encode(value.getId(), outStream, context.nested());
-      VarLongCoder.of().encode(value.getBlockOffset(), outStream, context.nested());
-      VarLongCoder.of().encode(value.getIndexOffset(), outStream, context.nested());
-    }
-
-    @Override
-    public IsmShard decode(
-        InputStream inStream, Coder.Context context) throws CoderException, IOException {
-      return IsmShard.of(
-          VarIntCoder.of().decode(inStream, context),
-          VarLongCoder.of().decode(inStream, context),
-          VarLongCoder.of().decode(inStream, context));
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      return true;
-    }
-  }
-
-  /**
-   * The prefix used before each key which contains the number of shared and unshared
-   * bytes from the previous key that was read. The key prefix along with the previous key
-   * and the unshared key bytes allows one to construct the current key by doing the following
-   * {@code currentKey = previousKey[0 : sharedBytes] + read(unsharedBytes)}.
-   *
-   * <p>The key prefix is encoded as:
-   * <ul>
-   *   <li>number of shared key bytes (variable length integer coding)</li>
-   *   <li>number of unshared key bytes (variable length integer coding)</li>
-   * </ul>
-   */
-  static class KeyPrefix {
-    private final int sharedKeySize;
-    private final int unsharedKeySize;
-
-    KeyPrefix(int sharedBytes, int unsharedBytes) {
-      this.sharedKeySize = sharedBytes;
-      this.unsharedKeySize = unsharedBytes;
-    }
-
-    public int getSharedKeySize() {
-      return sharedKeySize;
-    }
-
-    public int getUnsharedKeySize() {
-      return unsharedKeySize;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(sharedKeySize, unsharedKeySize);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == this) {
-        return true;
-      }
-      if (!(other instanceof KeyPrefix)) {
-        return false;
-      }
-      KeyPrefix keyPrefix = (KeyPrefix) other;
-      return sharedKeySize == keyPrefix.sharedKeySize
-          && unsharedKeySize == keyPrefix.unsharedKeySize;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("sharedKeySize", sharedKeySize)
-          .add("unsharedKeySize", unsharedKeySize)
-          .toString();
-    }
-  }
-
-  /** A {@link Coder} for {@link KeyPrefix}. */
-  static final class KeyPrefixCoder extends AtomicCoder<KeyPrefix> {
-    private static final KeyPrefixCoder INSTANCE = new KeyPrefixCoder();
-
-    @JsonCreator
-    public static KeyPrefixCoder of() {
-      return INSTANCE;
-    }
-
-    @Override
-    public void encode(KeyPrefix value, OutputStream outStream, Coder.Context context)
-        throws CoderException, IOException {
-      VarInt.encode(value.sharedKeySize, outStream);
-      VarInt.encode(value.unsharedKeySize, outStream);
-    }
-
-    @Override
-    public KeyPrefix decode(InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      return new KeyPrefix(VarInt.decodeInt(inStream), VarInt.decodeInt(inStream));
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      return true;
-    }
-
-    @Override
-    public boolean isRegisterByteSizeObserverCheap(KeyPrefix value, Coder.Context context) {
-      return true;
-    }
-
-    @Override
-    protected long getEncodedElementByteSize(KeyPrefix value, Coder.Context context)
-        throws Exception {
-      Preconditions.checkNotNull(value);
-      return VarInt.getLength(value.sharedKeySize) + VarInt.getLength(value.unsharedKeySize);
-    }
-  }
-
-  /**
-   * The footer stores the relevant information required to locate the index and bloom filter.
-   * It also stores a version byte and the number of keys stored.
-   *
-   * <p>The footer is encoded as the value containing:
-   * <ul>
-   *   <li>start of bloom filter offset (big endian long coding)</li>
-   *   <li>start of shard index position offset (big endian long coding)</li>
-   *   <li>number of keys in file (big endian long coding)</li>
-   *   <li>0x01 (version key as a single byte)</li>
-   * </ul>
-   */
-  static class Footer {
-    static final int LONG_BYTES = 8;
-    static final int FIXED_LENGTH = 3 * LONG_BYTES + 1;
-    static final byte VERSION = 2;
-
-    private final long indexPosition;
-    private final long bloomFilterPosition;
-    private final long numberOfKeys;
-
-    Footer(long indexPosition, long bloomFilterPosition, long numberOfKeys) {
-      this.indexPosition = indexPosition;
-      this.bloomFilterPosition = bloomFilterPosition;
-      this.numberOfKeys = numberOfKeys;
-    }
-
-    public long getIndexPosition() {
-      return indexPosition;
-    }
-
-    public long getBloomFilterPosition() {
-      return bloomFilterPosition;
-    }
-
-    public long getNumberOfKeys() {
-      return numberOfKeys;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == this) {
-        return true;
-      }
-      if (!(other instanceof Footer)) {
-        return false;
-      }
-      Footer footer = (Footer) other;
-      return indexPosition == footer.indexPosition
-          && bloomFilterPosition == footer.bloomFilterPosition
-          && numberOfKeys == footer.numberOfKeys;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(indexPosition, bloomFilterPosition, numberOfKeys);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("version", Footer.VERSION)
-          .add("indexPosition", indexPosition)
-          .add("bloomFilterPosition", bloomFilterPosition)
-          .add("numberOfKeys", numberOfKeys)
-          .toString();
-    }
-  }
-
-  /** A {@link Coder} for {@link Footer}. */
-  static final class FooterCoder extends AtomicCoder<Footer> {
-    private static final FooterCoder INSTANCE = new FooterCoder();
-
-    @JsonCreator
-    public static FooterCoder of() {
-      return INSTANCE;
-    }
-
-    @Override
-    public void encode(Footer value, OutputStream outStream, Coder.Context context)
-        throws CoderException, IOException {
-      DataOutputStream dataOut = new DataOutputStream(outStream);
-      dataOut.writeLong(value.indexPosition);
-      dataOut.writeLong(value.bloomFilterPosition);
-      dataOut.writeLong(value.numberOfKeys);
-      dataOut.write(Footer.VERSION);
-    }
-
-    @Override
-    public Footer decode(InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      DataInputStream dataIn = new DataInputStream(inStream);
-      Footer footer = new Footer(dataIn.readLong(), dataIn.readLong(), dataIn.readLong());
-      int version = dataIn.read();
-      if (version != Footer.VERSION) {
-        throw new IOException("Unknown version " + version + ". "
-            + "Only version 2 is currently supported.");
-      }
-      return footer;
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      return true;
-    }
-
-    @Override
-    public boolean isRegisterByteSizeObserverCheap(Footer value, Coder.Context context) {
-      return true;
-    }
-
-    @Override
-    protected long getEncodedElementByteSize(Footer value, Coder.Context context)
-        throws Exception {
-      return Footer.FIXED_LENGTH;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java
deleted file mode 100644
index 00ecfdf..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java
+++ /dev/null
@@ -1,25 +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.
- */
-/**
- * Implementation of the harness that runs on each Google Compute Engine instance to coordinate
- * execution of Pipeline code.
- */
-@ParametersAreNonnullByDefault
-package com.google.cloud.dataflow.sdk.runners.worker;
-
-import javax.annotation.ParametersAreNonnullByDefault;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java
deleted file mode 100644
index 8cd05d7..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java
+++ /dev/null
@@ -1,350 +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 com.google.cloud.dataflow.sdk.testing;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasItem;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.Serializer;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.UnownedInputStream;
-import com.google.cloud.dataflow.sdk.util.UnownedOutputStream;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Properties for use in {@link Coder} tests. These are implemented with junit assertions
- * rather than as predicates for the sake of error messages.
- *
- * <p>We serialize and deserialize the coder to make sure that any state information required by
- * the coder is preserved. This causes tests written such that coders that lose information during
- * serialization or change state during encoding/decoding will fail.
- */
-public class CoderProperties {
-
-  /**
-   * All the contexts, for use in test cases.
-   */
-   public static final List<Coder.Context> ALL_CONTEXTS = Arrays.asList(
-       Coder.Context.OUTER, Coder.Context.NESTED);
-
-  /**
-   * Verifies that for the given {@code Coder<T>}, and values of
-   * type {@code T}, if the values are equal then the encoded bytes are equal, in any
-   * {@code Coder.Context}.
-   */
-  public static <T> void coderDeterministic(
-      Coder<T> coder, T value1, T value2)
-      throws Exception {
-    for (Coder.Context context : ALL_CONTEXTS) {
-      coderDeterministicInContext(coder, context, value1, value2);
-    }
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<T>}, {@code Coder.Context}, and values of
-   * type {@code T}, if the values are equal then the encoded bytes are equal.
-   */
-  public static <T> void coderDeterministicInContext(
-      Coder<T> coder, Coder.Context context, T value1, T value2)
-      throws Exception {
-
-    try {
-      coder.verifyDeterministic();
-    } catch (NonDeterministicException e) {
-      fail("Expected that the coder is deterministic");
-    }
-    assertThat("Expected that the passed in values are equal()", value1, equalTo(value2));
-    assertThat(
-        encode(coder, context, value1),
-        equalTo(encode(coder, context, value2)));
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<T>},
-   * and value of type {@code T}, encoding followed by decoding yields an
-   * equal value of type {@code T}, in any {@code Coder.Context}.
-   */
-  public static <T> void coderDecodeEncodeEqual(
-      Coder<T> coder, T value)
-      throws Exception {
-    for (Coder.Context context : ALL_CONTEXTS) {
-      coderDecodeEncodeEqualInContext(coder, context, value);
-    }
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<T>}, {@code Coder.Context},
-   * and value of type {@code T}, encoding followed by decoding yields an
-   * equal value of type {@code T}.
-   */
-  public static <T> void coderDecodeEncodeEqualInContext(
-      Coder<T> coder, Coder.Context context, T value)
-      throws Exception {
-    assertThat(decodeEncode(coder, context, value), equalTo(value));
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<Collection<T>>},
-   * and value of type {@code Collection<T>}, encoding followed by decoding yields an
-   * equal value of type {@code Collection<T>}, in any {@code Coder.Context}.
-   */
-  public static <T, CollectionT extends Collection<T>> void coderDecodeEncodeContentsEqual(
-      Coder<CollectionT> coder, CollectionT value)
-      throws Exception {
-    for (Coder.Context context : ALL_CONTEXTS) {
-      coderDecodeEncodeContentsEqualInContext(coder, context, value);
-    }
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<Collection<T>>},
-   * and value of type {@code Collection<T>}, encoding followed by decoding yields an
-   * equal value of type {@code Collection<T>}, in the given {@code Coder.Context}.
-   */
-  @SuppressWarnings("unchecked")
-  public static <T, CollectionT extends Collection<T>> void coderDecodeEncodeContentsEqualInContext(
-      Coder<CollectionT> coder, Coder.Context context, CollectionT value)
-      throws Exception {
-    // Matchers.containsInAnyOrder() requires at least one element
-    Collection<T> result = decodeEncode(coder, context, value);
-    if (value.isEmpty()) {
-      assertThat(result, emptyIterable());
-    } else {
-      // This is the only Matchers.containInAnyOrder() overload that takes literal values
-      assertThat(result, containsInAnyOrder((T[]) value.toArray()));
-    }
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<Collection<T>>},
-   * and value of type {@code Collection<T>}, encoding followed by decoding yields an
-   * equal value of type {@code Collection<T>}, in any {@code Coder.Context}.
-   */
-  public static <T, IterableT extends Iterable<T>> void coderDecodeEncodeContentsInSameOrder(
-      Coder<IterableT> coder, IterableT value)
-      throws Exception {
-    for (Coder.Context context : ALL_CONTEXTS) {
-      CoderProperties.<T, IterableT>coderDecodeEncodeContentsInSameOrderInContext(
-          coder, context, value);
-    }
-  }
-
-  /**
-   * Verifies that for the given {@code Coder<Iterable<T>>},
-   * and value of type {@code Iterable<T>}, encoding followed by decoding yields an
-   * equal value of type {@code Collection<T>}, in the given {@code Coder.Context}.
-   */
-  @SuppressWarnings("unchecked")
-  public static <T, IterableT extends Iterable<T>> void
-      coderDecodeEncodeContentsInSameOrderInContext(
-          Coder<IterableT> coder, Coder.Context context, IterableT value)
-      throws Exception {
-    Iterable<T> result = decodeEncode(coder, context, value);
-    // Matchers.contains() requires at least one element
-    if (Iterables.isEmpty(value)) {
-      assertThat(result, emptyIterable());
-    } else {
-      // This is the only Matchers.contains() overload that takes literal values
-      assertThat(result, contains((T[]) Iterables.toArray(value, Object.class)));
-    }
-  }
-
-  public static <T> void coderSerializable(Coder<T> coder) {
-    SerializableUtils.ensureSerializable(coder);
-  }
-
-  public static <T> void coderConsistentWithEquals(
-      Coder<T> coder, T value1, T value2)
-      throws Exception {
-
-    for (Coder.Context context : ALL_CONTEXTS) {
-      CoderProperties.<T>coderConsistentWithEqualsInContext(coder, context, value1, value2);
-    }
-  }
-
-  public static <T> void coderConsistentWithEqualsInContext(
-      Coder<T> coder, Coder.Context context, T value1, T value2) throws Exception {
-
-    assertEquals(
-        value1.equals(value2),
-        Arrays.equals(
-            encode(coder, context, value1),
-            encode(coder, context, value2)));
-  }
-
-  public static <T> void coderHasEncodingId(Coder<T> coder, String encodingId) throws Exception {
-    assertThat(coder.getEncodingId(), equalTo(encodingId));
-    assertThat(Structs.getString(coder.asCloudObject(), PropertyNames.ENCODING_ID, ""),
-        equalTo(encodingId));
-  }
-
-  public static <T> void coderAllowsEncoding(Coder<T> coder, String encodingId) throws Exception {
-    assertThat(coder.getAllowedEncodings(), hasItem(encodingId));
-    assertThat(
-        String.format("Expected to find \"%s\" in property \"%s\" of %s",
-            encodingId, PropertyNames.ALLOWED_ENCODINGS, coder.asCloudObject()),
-        Structs.getStrings(
-            coder.asCloudObject(),
-            PropertyNames.ALLOWED_ENCODINGS,
-            Collections.<String>emptyList()),
-        hasItem(encodingId));
-  }
-
-  public static <T> void structuralValueConsistentWithEquals(
-      Coder<T> coder, T value1, T value2)
-      throws Exception {
-
-    for (Coder.Context context : ALL_CONTEXTS) {
-      CoderProperties.<T>structuralValueConsistentWithEqualsInContext(
-          coder, context, value1, value2);
-    }
-  }
-
-  public static <T> void structuralValueConsistentWithEqualsInContext(
-      Coder<T> coder, Coder.Context context, T value1, T value2) throws Exception {
-
-    assertEquals(
-        coder.structuralValue(value1).equals(coder.structuralValue(value2)),
-        Arrays.equals(
-            encode(coder, context, value1),
-            encode(coder, context, value2)));
-  }
-
-
-  private static final String DECODING_WIRE_FORMAT_MESSAGE =
-      "Decoded value from known wire format does not match expected value."
-      + " This probably means that this Coder no longer correctly decodes"
-      + " a prior wire format. Changing the wire formats this Coder can read"
-      + " should be avoided, as it is likely to cause breakage."
-      + " If you truly intend to change the backwards compatibility for this Coder "
-      + " then you must remove any now-unsupported encodings from getAllowedEncodings().";
-
-  public static <T> void coderDecodesBase64(Coder<T> coder, String base64Encoding, T value)
-      throws Exception {
-    assertThat(DECODING_WIRE_FORMAT_MESSAGE, CoderUtils.decodeFromBase64(coder, base64Encoding),
-        equalTo(value));
-  }
-
-  public static <T> void coderDecodesBase64(
-      Coder<T> coder, List<String> base64Encodings, List<T> values) throws Exception {
-    assertThat("List of base64 encodings has different size than List of values",
-        base64Encodings.size(), equalTo(values.size()));
-
-    for (int i = 0; i < base64Encodings.size(); i++) {
-      coderDecodesBase64(coder, base64Encodings.get(i), values.get(i));
-    }
-  }
-
-  private static final String ENCODING_WIRE_FORMAT_MESSAGE =
-      "Encoded value does not match expected wire format."
-      + " Changing the wire format should be avoided, as it is likely to cause breakage."
-      + " If you truly intend to change the wire format for this Coder "
-      + " then you must update getEncodingId() to a new value and add any supported"
-      + " prior formats to getAllowedEncodings()."
-      + " See com.google.cloud.dataflow.sdk.coders.PrintBase64Encoding for how to generate"
-      + " new test data.";
-
-  public static <T> void coderEncodesBase64(Coder<T> coder, T value, String base64Encoding)
-      throws Exception {
-    assertThat(ENCODING_WIRE_FORMAT_MESSAGE, CoderUtils.encodeToBase64(coder, value),
-        equalTo(base64Encoding));
-  }
-
-  public static <T> void coderEncodesBase64(
-      Coder<T> coder, List<T> values, List<String> base64Encodings) throws Exception {
-    assertThat("List of base64 encodings has different size than List of values",
-        base64Encodings.size(), equalTo(values.size()));
-
-    for (int i = 0; i < base64Encodings.size(); i++) {
-      coderEncodesBase64(coder, values.get(i), base64Encodings.get(i));
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public static <T, IterableT extends Iterable<T>> void coderDecodesBase64ContentsEqual(
-      Coder<IterableT> coder, String base64Encoding, IterableT expected) throws Exception {
-
-    IterableT result = CoderUtils.decodeFromBase64(coder, base64Encoding);
-    if (Iterables.isEmpty(expected)) {
-      assertThat(ENCODING_WIRE_FORMAT_MESSAGE, result, emptyIterable());
-    } else {
-      assertThat(ENCODING_WIRE_FORMAT_MESSAGE, result,
-          containsInAnyOrder((T[]) Iterables.toArray(expected, Object.class)));
-    }
-  }
-
-  public static <T, IterableT extends Iterable<T>> void coderDecodesBase64ContentsEqual(
-      Coder<IterableT> coder, List<String> base64Encodings, List<IterableT> expected)
-          throws Exception {
-    assertThat("List of base64 encodings has different size than List of values",
-        base64Encodings.size(), equalTo(expected.size()));
-
-    for (int i = 0; i < base64Encodings.size(); i++) {
-      coderDecodesBase64ContentsEqual(coder, base64Encodings.get(i), expected.get(i));
-    }
-  }
-
-  //////////////////////////////////////////////////////////////////////////
-
-  @VisibleForTesting
-  static <T> byte[] encode(
-      Coder<T> coder, Coder.Context context, T value) throws CoderException, IOException {
-    @SuppressWarnings("unchecked")
-    Coder<T> deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class);
-
-    ByteArrayOutputStream os = new ByteArrayOutputStream();
-    deserializedCoder.encode(value, new UnownedOutputStream(os), context);
-    return os.toByteArray();
-  }
-
-  @VisibleForTesting
-  static <T> T decode(
-      Coder<T> coder, Coder.Context context, byte[] bytes) throws CoderException, IOException {
-    @SuppressWarnings("unchecked")
-    Coder<T> deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class);
-
-    ByteArrayInputStream is = new ByteArrayInputStream(bytes);
-    return deserializedCoder.decode(new UnownedInputStream(is), context);
-  }
-
-  private static <T> T decodeEncode(Coder<T> coder, Coder.Context context, T value)
-      throws CoderException, IOException {
-    return decode(coder, context, encode(coder, context, value));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/PAssert.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/PAssert.java
deleted file mode 100644
index aebce42..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/PAssert.java
+++ /dev/null
@@ -1,825 +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 com.google.cloud.dataflow.sdk.testing;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.MapCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-
-/**
- * An assertion on the contents of a {@link PCollection}
- * incorporated into the pipeline.  Such an assertion
- * can be checked no matter what kind of {@link PipelineRunner} is
- * used.
- *
- * <p>Note that the {@code PAssert} call must precede the call
- * to {@link Pipeline#run}.
- *
- * <p>Examples of use:
- * <pre>{@code
- * Pipeline p = TestPipeline.create();
- * ...
- * PCollection<String> output =
- *      input
- *      .apply(ParDo.of(new TestDoFn()));
- * PAssert.that(output)
- *     .containsInAnyOrder("out1", "out2", "out3");
- * ...
- * PCollection<Integer> ints = ...
- * PCollection<Integer> sum =
- *     ints
- *     .apply(Combine.globally(new SumInts()));
- * PAssert.that(sum)
- *     .is(42);
- * ...
- * p.run();
- * }</pre>
- *
- * <p>JUnit and Hamcrest must be linked in by any code that uses PAssert.
- */
-public class PAssert {
-
-  private static final Logger LOG = LoggerFactory.getLogger(PAssert.class);
-
-  static final String SUCCESS_COUNTER = "PAssertSuccess";
-  static final String FAILURE_COUNTER = "PAssertFailure";
-
-  private static int assertCount = 0;
-
-  // Do not instantiate.
-  private PAssert() {}
-
-  /**
-   * Constructs an {@link IterableAssert} for the elements of the provided
-   * {@link PCollection}.
-   */
-  public static <T> IterableAssert<T> that(PCollection<T> actual) {
-    return new IterableAssert<>(
-        new CreateActual<T, Iterable<T>>(actual, View.<T>asIterable()),
-         actual.getPipeline())
-         .setCoder(actual.getCoder());
-  }
-
-  /**
-   * Constructs an {@link IterableAssert} for the value of the provided
-   * {@link PCollection} which must contain a single {@code Iterable<T>}
-   * value.
-   */
-  public static <T> IterableAssert<T>
-      thatSingletonIterable(PCollection<? extends Iterable<T>> actual) {
-
-    List<? extends Coder<?>> maybeElementCoder = actual.getCoder().getCoderArguments();
-    Coder<T> tCoder;
-    try {
-      @SuppressWarnings("unchecked")
-      Coder<T> tCoderTmp = (Coder<T>) Iterables.getOnlyElement(maybeElementCoder);
-      tCoder = tCoderTmp;
-    } catch (NoSuchElementException | IllegalArgumentException exc) {
-      throw new IllegalArgumentException(
-          "PAssert.<T>thatSingletonIterable requires a PCollection<Iterable<T>>"
-              + " with a Coder<Iterable<T>> where getCoderArguments() yields a"
-              + " single Coder<T> to apply to the elements.");
-    }
-
-    @SuppressWarnings("unchecked") // Safe covariant cast
-    PCollection<Iterable<T>> actualIterables = (PCollection<Iterable<T>>) actual;
-
-    return new IterableAssert<>(
-        new CreateActual<Iterable<T>, Iterable<T>>(
-            actualIterables, View.<Iterable<T>>asSingleton()),
-        actual.getPipeline())
-        .setCoder(tCoder);
-  }
-
-  /**
-   * Constructs an {@link IterableAssert} for the value of the provided
-   * {@code PCollectionView PCollectionView<Iterable<T>>}.
-   */
-  public static <T> IterableAssert<T> thatIterable(PCollectionView<Iterable<T>> actual) {
-    return new IterableAssert<>(new PreExisting<Iterable<T>>(actual), actual.getPipeline());
-  }
-
-  /**
-   * Constructs a {@link SingletonAssert} for the value of the provided
-   * {@code PCollection PCollection<T>}, which must be a singleton.
-   */
-  public static <T> SingletonAssert<T> thatSingleton(PCollection<T> actual) {
-    return new SingletonAssert<>(
-        new CreateActual<T, T>(actual, View.<T>asSingleton()), actual.getPipeline())
-        .setCoder(actual.getCoder());
-  }
-
-  /**
-   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}.
-   *
-   * <p>Note that the actual value must be coded by a {@link KvCoder},
-   * not just any {@code Coder<K, V>}.
-   */
-  public static <K, V> SingletonAssert<Map<K, Iterable<V>>>
-      thatMultimap(PCollection<KV<K, V>> actual) {
-    @SuppressWarnings("unchecked")
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
-
-    return new SingletonAssert<>(
-        new CreateActual<>(actual, View.<K, V>asMultimap()), actual.getPipeline())
-        .setCoder(MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())));
-  }
-
-  /**
-   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection},
-   * which must have at most one value per key.
-   *
-   * <p>Note that the actual value must be coded by a {@link KvCoder},
-   * not just any {@code Coder<K, V>}.
-   */
-  public static <K, V> SingletonAssert<Map<K, V>> thatMap(PCollection<KV<K, V>> actual) {
-    @SuppressWarnings("unchecked")
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
-
-    return new SingletonAssert<>(
-        new CreateActual<>(actual, View.<K, V>asMap()), actual.getPipeline())
-        .setCoder(MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()));
-  }
-
-  ////////////////////////////////////////////////////////////
-
-  /**
-   * An assertion about the contents of a {@link PCollectionView} yielding an {@code Iterable<T>}.
-   */
-  public static class IterableAssert<T> implements Serializable {
-    private final Pipeline pipeline;
-    private final PTransform<PBegin, PCollectionView<Iterable<T>>> createActual;
-    private Optional<Coder<T>> coder;
-
-    protected IterableAssert(
-        PTransform<PBegin, PCollectionView<Iterable<T>>> createActual, Pipeline pipeline) {
-      this.createActual = createActual;
-      this.pipeline = pipeline;
-      this.coder = Optional.absent();
-    }
-
-    /**
-     * Sets the coder to use for elements of type {@code T}, as needed for internal purposes.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> setCoder(Coder<T> coderOrNull) {
-      this.coder = Optional.fromNullable(coderOrNull);
-      return this;
-    }
-
-    /**
-     * Gets the coder, which may yet be absent.
-     */
-    public Coder<T> getCoder() {
-      if (coder.isPresent()) {
-        return coder.get();
-      } else {
-        throw new IllegalStateException(
-            "Attempting to access the coder of an IterableAssert"
-                + " that has not been set yet.");
-      }
-    }
-
-    /**
-     * Applies a {@link SerializableFunction} to check the elements of the {@code Iterable}.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> satisfies(SerializableFunction<Iterable<T>, Void> checkerFn) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new OneSideInputAssert<Iterable<T>>(createActual, checkerFn));
-      return this;
-    }
-
-    /**
-     * Applies a {@link SerializableFunction} to check the elements of the {@code Iterable}.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> satisfies(
-        AssertRelation<Iterable<T>, Iterable<T>> relation,
-        final Iterable<T> expectedElements) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new TwoSideInputAssert<Iterable<T>, Iterable<T>>(
-              createActual,
-              new CreateExpected<T, Iterable<T>>(expectedElements, coder, View.<T>asIterable()),
-              relation));
-
-      return this;
-    }
-
-    /**
-     * Applies a {@link SerializableMatcher} to check the elements of the {@code Iterable}.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    IterableAssert<T> satisfies(final SerializableMatcher<Iterable<? extends T>> matcher) {
-      // Safe covariant cast. Could be elided by changing a lot of this file to use
-      // more flexible bounds.
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      SerializableFunction<Iterable<T>, Void> checkerFn =
-        (SerializableFunction) new MatcherCheckerFn<>(matcher);
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new OneSideInputAssert<Iterable<T>>(createActual, checkerFn));
-      return this;
-    }
-
-    private static class MatcherCheckerFn<T> implements SerializableFunction<T, Void> {
-      private SerializableMatcher<T> matcher;
-
-      public MatcherCheckerFn(SerializableMatcher<T> matcher) {
-        this.matcher = matcher;
-      }
-
-      @Override
-      public Void apply(T actual) {
-        assertThat(actual, matcher);
-        return null;
-      }
-    }
-
-    /**
-     * Checks that the {@code Iterable} is empty.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> empty() {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), Collections.<T>emptyList());
-    }
-
-    /**
-     * @throws UnsupportedOperationException always
-     * @deprecated {@link Object#equals(Object)} is not supported on PAssert objects.
-     *    If you meant to test object equality, use a variant of {@link #containsInAnyOrder}
-     *    instead.
-     */
-    @Deprecated
-    @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException(
-          "If you meant to test object equality, use .containsInAnyOrder instead.");
-    }
-
-    /**
-     * @throws UnsupportedOperationException always.
-     * @deprecated {@link Object#hashCode()} is not supported on PAssert objects.
-     */
-    @Deprecated
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException(
-          String.format("%s.hashCode() is not supported.", IterableAssert.class.getSimpleName()));
-    }
-
-    /**
-     * Checks that the {@code Iterable} contains the expected elements, in any
-     * order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
-    }
-
-    /**
-     * Checks that the {@code Iterable} contains the expected elements, in any
-     * order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    @SafeVarargs
-    public final IterableAssert<T> containsInAnyOrder(T... expectedElements) {
-      return satisfies(
-        new AssertContainsInAnyOrderRelation<T>(),
-        Arrays.asList(expectedElements));
-    }
-
-    /**
-     * Checks that the {@code Iterable} contains elements that match the provided matchers,
-     * in any order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    @SafeVarargs
-    final IterableAssert<T> containsInAnyOrder(
-        SerializableMatcher<? super T>... elementMatchers) {
-      return satisfies(SerializableMatchers.<T>containsInAnyOrder(elementMatchers));
-    }
-  }
-
-  /**
-   * An assertion about the single value of type {@code T}
-   * associated with a {@link PCollectionView}.
-   */
-  public static class SingletonAssert<T> implements Serializable {
-    private final Pipeline pipeline;
-    private final CreateActual<?, T> createActual;
-    private Optional<Coder<T>> coder;
-
-    protected SingletonAssert(
-        CreateActual<?, T> createActual, Pipeline pipeline) {
-      this.pipeline = pipeline;
-      this.createActual = createActual;
-      this.coder = Optional.absent();
-    }
-
-    /**
-     * Always throws an {@link UnsupportedOperationException}: users are probably looking for
-     * {@link #isEqualTo}.
-     */
-    @Deprecated
-    @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException(
-          String.format(
-              "tests for Java equality of the %s object, not the PCollection in question. "
-                  + "Call a test method, such as isEqualTo.",
-              getClass().getSimpleName()));
-    }
-
-    /**
-     * @throws UnsupportedOperationException always.
-     * @deprecated {@link Object#hashCode()} is not supported on PAssert objects.
-     */
-    @Deprecated
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException(
-          String.format("%s.hashCode() is not supported.", SingletonAssert.class.getSimpleName()));
-    }
-
-    /**
-     * Sets the coder to use for elements of type {@code T}, as needed
-     * for internal purposes.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public SingletonAssert<T> setCoder(Coder<T> coderOrNull) {
-      this.coder = Optional.fromNullable(coderOrNull);
-      return this;
-    }
-
-    /**
-     * Gets the coder, which may yet be absent.
-     */
-    public Coder<T> getCoder() {
-      if (coder.isPresent()) {
-        return coder.get();
-      } else {
-        throw new IllegalStateException(
-            "Attempting to access the coder of an IterableAssert that has not been set yet.");
-      }
-    }
-
-    /**
-     * Applies a {@link SerializableFunction} to check the value of this
-     * {@code SingletonAssert}'s view.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> satisfies(SerializableFunction<T, Void> checkerFn) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++), new OneSideInputAssert<T>(createActual, checkerFn));
-      return this;
-    }
-
-    /**
-     * Applies an {@link AssertRelation} to check the provided relation against the
-     * value of this assert and the provided expected value.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> satisfies(
-        AssertRelation<T, T> relation,
-        final T expectedValue) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new TwoSideInputAssert<T, T>(
-              createActual,
-              new CreateExpected<T, T>(Arrays.asList(expectedValue), coder, View.<T>asSingleton()),
-              relation));
-
-      return this;
-    }
-
-    /**
-     * Checks that the value of this {@code SingletonAssert}'s view is equal
-     * to the expected value.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> isEqualTo(T expectedValue) {
-      return satisfies(new AssertIsEqualToRelation<T>(), expectedValue);
-    }
-
-    /**
-     * Checks that the value of this {@code SingletonAssert}'s view is not equal
-     * to the expected value.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> notEqualTo(T expectedValue) {
-      return satisfies(new AssertNotEqualToRelation<T>(), expectedValue);
-    }
-
-    /**
-     * Checks that the value of this {@code SingletonAssert}'s view is equal to
-     * the expected value.
-     *
-     * @deprecated replaced by {@link #isEqualTo}
-     */
-    @Deprecated
-    public SingletonAssert<T> is(T expectedValue) {
-      return isEqualTo(expectedValue);
-    }
-
-  }
-
-  ////////////////////////////////////////////////////////////////////////
-
-  private static class CreateActual<T, ActualT>
-      extends PTransform<PBegin, PCollectionView<ActualT>> {
-
-    private final transient PCollection<T> actual;
-    private final transient PTransform<PCollection<T>, PCollectionView<ActualT>> actualView;
-
-    private CreateActual(PCollection<T> actual,
-        PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
-      this.actual = actual;
-      this.actualView = actualView;
-    }
-
-    @Override
-    public PCollectionView<ActualT> apply(PBegin input) {
-      final Coder<T> coder = actual.getCoder();
-      return actual
-          .apply(Window.<T>into(new GlobalWindows()))
-          .apply(ParDo.of(new DoFn<T, T>() {
-            @Override
-            public void processElement(ProcessContext context) throws CoderException {
-              context.output(CoderUtils.clone(coder, context.element()));
-            }
-          }))
-          .apply(actualView);
-    }
-  }
-
-  private static class CreateExpected<T, ExpectedT>
-      extends PTransform<PBegin, PCollectionView<ExpectedT>> {
-
-    private final Iterable<T> elements;
-    private final Optional<Coder<T>> coder;
-    private final transient PTransform<PCollection<T>, PCollectionView<ExpectedT>> view;
-
-    private CreateExpected(Iterable<T> elements, Optional<Coder<T>> coder,
-        PTransform<PCollection<T>, PCollectionView<ExpectedT>> view) {
-      this.elements = elements;
-      this.coder = coder;
-      this.view = view;
-    }
-
-    @Override
-    public PCollectionView<ExpectedT> apply(PBegin input) {
-      Create.Values<T> createTransform = Create.<T>of(elements);
-      if (coder.isPresent()) {
-        createTransform = createTransform.withCoder(coder.get());
-      }
-      return input.apply(createTransform).apply(view);
-    }
-  }
-
-  private static class PreExisting<T> extends PTransform<PBegin, PCollectionView<T>> {
-
-    private final PCollectionView<T> view;
-
-    private PreExisting(PCollectionView<T> view) {
-      this.view = view;
-    }
-
-    @Override
-    public PCollectionView<T> apply(PBegin input) {
-      return view;
-    }
-  }
-
-  /**
-   * An assertion checker that takes a single
-   * {@link PCollectionView PCollectionView&lt;ActualT&gt;}
-   * and an assertion over {@code ActualT}, and checks it within a dataflow
-   * pipeline.
-   *
-   * <p>Note that the entire assertion must be serializable. If
-   * you need to make assertions involving multiple inputs
-   * that are each not serializable, use TwoSideInputAssert.
-   *
-   * <p>This is generally useful for assertion functions that
-   * are serializable but whose underlying data may not have a coder.
-   */
-  static class OneSideInputAssert<ActualT>
-      extends PTransform<PBegin, PDone> implements Serializable {
-    private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
-    private final SerializableFunction<ActualT, Void> checkerFn;
-
-    public OneSideInputAssert(
-        PTransform<PBegin, PCollectionView<ActualT>> createActual,
-        SerializableFunction<ActualT, Void> checkerFn) {
-      this.createActual = createActual;
-      this.checkerFn = checkerFn;
-    }
-
-    @Override
-    public PDone apply(PBegin input) {
-      final PCollectionView<ActualT> actual = input.apply("CreateActual", createActual);
-
-      input
-          .apply(Create.<Void>of((Void) null).withCoder(VoidCoder.of()))
-          .apply(ParDo.named("RunChecks").withSideInputs(actual)
-              .of(new CheckerDoFn<>(checkerFn, actual)));
-
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  /**
-   * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of
-   * a {@link PCollectionView}, and adjusts counters and thrown exceptions for use in testing.
-   */
-  private static class CheckerDoFn<ActualT> extends DoFn<Void, Void> {
-    private final SerializableFunction<ActualT, Void> checkerFn;
-    private final Aggregator<Integer, Integer> success =
-        createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
-    private final Aggregator<Integer, Integer> failure =
-        createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
-    private final PCollectionView<ActualT> actual;
-
-    private CheckerDoFn(
-        SerializableFunction<ActualT, Void> checkerFn,
-        PCollectionView<ActualT> actual) {
-      this.checkerFn = checkerFn;
-      this.actual = actual;
-    }
-
-    @Override
-    public void processElement(ProcessContext c) {
-      try {
-        ActualT actualContents = c.sideInput(actual);
-        checkerFn.apply(actualContents);
-        success.addValue(1);
-      } catch (Throwable t) {
-        LOG.error("PAssert failed expectations.", t);
-        failure.addValue(1);
-        // TODO: allow for metrics to propagate on failure when running a streaming pipeline
-        if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
-          throw t;
-        }
-      }
-    }
-  }
-
-  /**
-   * An assertion checker that takes a {@link PCollectionView PCollectionView&lt;ActualT&gt;},
-   * a {@link PCollectionView PCollectionView&lt;ExpectedT&gt;}, a relation
-   * over {@code A} and {@code B}, and checks that the relation holds
-   * within a dataflow pipeline.
-   *
-   * <p>This is useful when either/both of {@code A} and {@code B}
-   * are not serializable, but have coders (provided
-   * by the underlying {@link PCollection}s).
-   */
-  static class TwoSideInputAssert<ActualT, ExpectedT>
-      extends PTransform<PBegin, PDone> implements Serializable {
-
-    private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
-    private final transient PTransform<PBegin, PCollectionView<ExpectedT>> createExpected;
-    private final AssertRelation<ActualT, ExpectedT> relation;
-
-    protected TwoSideInputAssert(
-        PTransform<PBegin, PCollectionView<ActualT>> createActual,
-        PTransform<PBegin, PCollectionView<ExpectedT>> createExpected,
-        AssertRelation<ActualT, ExpectedT> relation) {
-      this.createActual = createActual;
-      this.createExpected = createExpected;
-      this.relation = relation;
-    }
-
-    @Override
-    public PDone apply(PBegin input) {
-      final PCollectionView<ActualT> actual = input.apply("CreateActual", createActual);
-      final PCollectionView<ExpectedT> expected = input.apply("CreateExpected", createExpected);
-
-      input
-          .apply(Create.<Void>of((Void) null).withCoder(VoidCoder.of()))
-          .apply(ParDo.named("RunChecks").withSideInputs(actual, expected)
-              .of(new CheckerDoFn<>(relation, actual, expected)));
-
-      return PDone.in(input.getPipeline());
-    }
-
-    private static class CheckerDoFn<ActualT, ExpectedT> extends DoFn<Void, Void> {
-      private final Aggregator<Integer, Integer> success =
-          createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
-      private final Aggregator<Integer, Integer> failure =
-          createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
-      private final AssertRelation<ActualT, ExpectedT> relation;
-      private final PCollectionView<ActualT> actual;
-      private final PCollectionView<ExpectedT> expected;
-
-      private CheckerDoFn(AssertRelation<ActualT, ExpectedT> relation,
-          PCollectionView<ActualT> actual, PCollectionView<ExpectedT> expected) {
-        this.relation = relation;
-        this.actual = actual;
-        this.expected = expected;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) {
-        try {
-          ActualT actualContents = c.sideInput(actual);
-          ExpectedT expectedContents = c.sideInput(expected);
-          relation.assertFor(expectedContents).apply(actualContents);
-          success.addValue(1);
-        } catch (Throwable t) {
-          LOG.error("PAssert failed expectations.", t);
-          failure.addValue(1);
-          // TODO: allow for metrics to propagate on failure when running a streaming pipeline
-          if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
-            throw t;
-          }
-        }
-      }
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * A {@link SerializableFunction} that verifies that an actual value is equal to an
-   * expected value.
-   */
-  private static class AssertIsEqualTo<T> implements SerializableFunction<T, Void> {
-    private T expected;
-
-    public AssertIsEqualTo(T expected) {
-      this.expected = expected;
-    }
-
-    @Override
-    public Void apply(T actual) {
-      assertThat(actual, equalTo(expected));
-      return null;
-    }
-  }
-
-  /**
-   * A {@link SerializableFunction} that verifies that an actual value is not equal to an
-   * expected value.
-   */
-  private static class AssertNotEqualTo<T> implements SerializableFunction<T, Void> {
-    private T expected;
-
-    public AssertNotEqualTo(T expected) {
-      this.expected = expected;
-    }
-
-    @Override
-    public Void apply(T actual) {
-      assertThat(actual, not(equalTo(expected)));
-      return null;
-    }
-  }
-
-  /**
-   * A {@link SerializableFunction} that verifies that an {@code Iterable} contains
-   * expected items in any order.
-   */
-  private static class AssertContainsInAnyOrder<T>
-      implements SerializableFunction<Iterable<T>, Void> {
-    private T[] expected;
-
-    @SafeVarargs
-    public AssertContainsInAnyOrder(T... expected) {
-      this.expected = expected;
-    }
-
-    @SuppressWarnings("unchecked")
-    public AssertContainsInAnyOrder(Collection<T> expected) {
-      this((T[]) expected.toArray());
-    }
-
-    public AssertContainsInAnyOrder(Iterable<T> expected) {
-      this(Lists.<T>newArrayList(expected));
-    }
-
-    @Override
-    public Void apply(Iterable<T> actual) {
-      assertThat(actual, containsInAnyOrder(expected));
-      return null;
-    }
-  }
-
-  ////////////////////////////////////////////////////////////
-
-  /**
-   * A binary predicate between types {@code Actual} and {@code Expected}.
-   * Implemented as a method {@code assertFor(Expected)} which returns
-   * a {@code SerializableFunction<Actual, Void>}
-   * that should verify the assertion..
-   */
-  private static interface AssertRelation<ActualT, ExpectedT> extends Serializable {
-    public SerializableFunction<ActualT, Void> assertFor(ExpectedT input);
-  }
-
-  /**
-   * An {@link AssertRelation} implementing the binary predicate that two objects are equal.
-   */
-  private static class AssertIsEqualToRelation<T>
-      implements AssertRelation<T, T> {
-    @Override
-    public SerializableFunction<T, Void> assertFor(T expected) {
-      return new AssertIsEqualTo<T>(expected);
-    }
-  }
-
-  /**
-   * An {@link AssertRelation} implementing the binary predicate that two objects are not equal.
-   */
-  private static class AssertNotEqualToRelation<T>
-      implements AssertRelation<T, T> {
-    @Override
-    public SerializableFunction<T, Void> assertFor(T expected) {
-      return new AssertNotEqualTo<T>(expected);
-    }
-  }
-
-  /**
-   * An {@code AssertRelation} implementing the binary predicate that two collections are equal
-   * modulo reordering.
-   */
-  private static class AssertContainsInAnyOrderRelation<T>
-      implements AssertRelation<Iterable<T>, Iterable<T>> {
-    @Override
-    public SerializableFunction<Iterable<T>, Void> assertFor(Iterable<T> expectedElements) {
-      return new AssertContainsInAnyOrder<T>(expectedElements);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java
deleted file mode 100644
index ea14774..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.testing;
-
-/**
- * Category tag for validation tests which utilize {@link TestPipeline} for execution and
- * {@link PAssert} for validation. Example usage:
- * <pre><code>
- *     {@literal @}Test
- *     {@literal @}Category(RunnableOnService.class)
- *     public void testParDo() {
- *       Pipeline p = TestPipeline.create();
- *       p.apply(...);
- *       PAssert.that(p);
- *       p.run();
- *     }
- * </code></pre>
- */
-public interface RunnableOnService {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatcher.java
deleted file mode 100644
index 238e0d1..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/SerializableMatcher.java
+++ /dev/null
@@ -1,37 +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 com.google.cloud.dataflow.sdk.testing;
-
-import org.hamcrest.Matcher;
-
-import java.io.Serializable;
-
-/**
- * A {@link Matcher} that is also {@link Serializable}.
- *
- * <p>Such matchers can be used with {@link PAssert}, which builds Dataflow pipelines
- * such that these matchers may be serialized and executed remotely.
- *
- * <p>To create a {@code SerializableMatcher}, extend {@link org.hamcrest.BaseMatcher}
- * and also implement this interface.
- *
- * @param <T> The type of value matched.
- */
-interface SerializableMatcher<T> extends Matcher<T>, Serializable {
-}
-


[49/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java
deleted file mode 100644
index 35c8655..0000000
--- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java
+++ /dev/null
@@ -1,153 +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 com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-
-/**
- * This test Outer Left Join functionality.
- */
-public class OuterLeftJoinTest {
-
-  Pipeline p;
-  List<KV<String, Long>> leftListOfKv;
-  List<KV<String, String>> listRightOfKv;
-  List<KV<String, KV<Long, String>>> expectedResult;
-
-  @Before
-  public void setup() {
-
-    p = TestPipeline.create();
-    leftListOfKv = new ArrayList<>();
-    listRightOfKv = new ArrayList<>();
-
-    expectedResult = new ArrayList<>();
-  }
-
-  @Test
-  public void testJoinOneToOneMapping() {
-    leftListOfKv.add(KV.of("Key1", 5L));
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key1", "foo"));
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
-      leftCollection, rightCollection, "");
-
-    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinOneToManyMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    listRightOfKv.add(KV.of("Key2", "gazonk"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
-      leftCollection, rightCollection, "");
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinManyToOneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    leftListOfKv.add(KV.of("Key2", 6L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
-      leftCollection, rightCollection, "");
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinOneToNoneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key3", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
-      leftCollection, rightCollection, "");
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-    p.run();
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinLeftCollectionNull() {
-    Join.leftOuterJoin(null, p.apply(Create.of(listRightOfKv)), "");
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinRightCollectionNull() {
-    Join.leftOuterJoin(p.apply(Create.of(leftListOfKv)), null, "");
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinNullValueIsNull() {
-    Join.leftOuterJoin(
-        p.apply("CreateLeft", Create.of(leftListOfKv)),
-        p.apply("CreateRight", Create.of(listRightOfKv)),
-        null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java
deleted file mode 100644
index 9cc6785..0000000
--- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java
+++ /dev/null
@@ -1,153 +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 com.google.cloud.dataflow.contrib.joinlibrary;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-
-/**
- * This test Outer Right Join functionality.
- */
-public class OuterRightJoinTest {
-
-  Pipeline p;
-  List<KV<String, Long>> leftListOfKv;
-  List<KV<String, String>> listRightOfKv;
-  List<KV<String, KV<Long, String>>> expectedResult;
-
-  @Before
-  public void setup() {
-
-    p = TestPipeline.create();
-    leftListOfKv = new ArrayList<>();
-    listRightOfKv = new ArrayList<>();
-
-    expectedResult = new ArrayList<>();
-  }
-
-  @Test
-  public void testJoinOneToOneMapping() {
-    leftListOfKv.add(KV.of("Key1", 5L));
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key1", "foo"));
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
-      leftCollection, rightCollection, -1L);
-
-    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinOneToManyMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    listRightOfKv.add(KV.of("Key2", "gazonk"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
-      leftCollection, rightCollection, -1L);
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinManyToOneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    leftListOfKv.add(KV.of("Key2", 6L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key2", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
-      leftCollection, rightCollection, -1L);
-
-    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
-    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-
-    p.run();
-  }
-
-  @Test
-  public void testJoinNoneToOneMapping() {
-    leftListOfKv.add(KV.of("Key2", 4L));
-    PCollection<KV<String, Long>> leftCollection = p
-        .apply("CreateLeft", Create.of(leftListOfKv));
-
-    listRightOfKv.add(KV.of("Key3", "bar"));
-    PCollection<KV<String, String>> rightCollection = p
-        .apply("CreateRight", Create.of(listRightOfKv));
-
-    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
-      leftCollection, rightCollection, -1L);
-
-    expectedResult.add(KV.of("Key3", KV.of(-1L, "bar")));
-    PAssert.that(output).containsInAnyOrder(expectedResult);
-    p.run();
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinLeftCollectionNull() {
-    Join.rightOuterJoin(null, p.apply(Create.of(listRightOfKv)), "");
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinRightCollectionNull() {
-    Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), null, -1L);
-  }
-
-  @Test(expected = NullPointerException.class)
-  public void testJoinNullValueIsNull() {
-    Join.rightOuterJoin(
-        p.apply("CreateLeft", Create.of(leftListOfKv)),
-        p.apply("CreateRight", Create.of(listRightOfKv)),
-        null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
new file mode 100644
index 0000000..c479a65
--- /dev/null
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/InnerJoinTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.joinlibrary;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This test Inner Join functionality.
+ */
+public class InnerJoinTest {
+
+  Pipeline p;
+  List<KV<String, Long>> leftListOfKv;
+  List<KV<String, String>> listRightOfKv;
+  List<KV<String, KV<Long, String>>> expectedResult;
+
+  @Before
+  public void setup() {
+
+    p = TestPipeline.create();
+    leftListOfKv = new ArrayList<>();
+    listRightOfKv = new ArrayList<>();
+
+    expectedResult = new ArrayList<>();
+  }
+
+  @Test
+  public void testJoinOneToOneMapping() {
+    leftListOfKv.add(KV.of("Key1", 5L));
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection =
+        p.apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key1", "foo"));
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection =
+        p.apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
+      leftCollection, rightCollection);
+
+    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinOneToManyMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    listRightOfKv.add(KV.of("Key2", "gazonk"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
+      leftCollection, rightCollection);
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinManyToOneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    leftListOfKv.add(KV.of("Key2", 6L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
+      leftCollection, rightCollection);
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinNoneToNoneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key3", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.innerJoin(
+      leftCollection, rightCollection);
+
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+    p.run();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinLeftCollectionNull() {
+    Join.innerJoin(null, p.apply(Create.of(listRightOfKv)));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinRightCollectionNull() {
+    Join.innerJoin(p.apply(Create.of(leftListOfKv)), null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
new file mode 100644
index 0000000..35c8655
--- /dev/null
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterLeftJoinTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.joinlibrary;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * This test Outer Left Join functionality.
+ */
+public class OuterLeftJoinTest {
+
+  Pipeline p;
+  List<KV<String, Long>> leftListOfKv;
+  List<KV<String, String>> listRightOfKv;
+  List<KV<String, KV<Long, String>>> expectedResult;
+
+  @Before
+  public void setup() {
+
+    p = TestPipeline.create();
+    leftListOfKv = new ArrayList<>();
+    listRightOfKv = new ArrayList<>();
+
+    expectedResult = new ArrayList<>();
+  }
+
+  @Test
+  public void testJoinOneToOneMapping() {
+    leftListOfKv.add(KV.of("Key1", 5L));
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key1", "foo"));
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
+      leftCollection, rightCollection, "");
+
+    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinOneToManyMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    listRightOfKv.add(KV.of("Key2", "gazonk"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
+      leftCollection, rightCollection, "");
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinManyToOneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    leftListOfKv.add(KV.of("Key2", 6L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
+      leftCollection, rightCollection, "");
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinOneToNoneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key3", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.leftOuterJoin(
+      leftCollection, rightCollection, "");
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+    p.run();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinLeftCollectionNull() {
+    Join.leftOuterJoin(null, p.apply(Create.of(listRightOfKv)), "");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinRightCollectionNull() {
+    Join.leftOuterJoin(p.apply(Create.of(leftListOfKv)), null, "");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinNullValueIsNull() {
+    Join.leftOuterJoin(
+        p.apply("CreateLeft", Create.of(leftListOfKv)),
+        p.apply("CreateRight", Create.of(listRightOfKv)),
+        null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
----------------------------------------------------------------------
diff --git a/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
new file mode 100644
index 0000000..9cc6785
--- /dev/null
+++ b/contrib/join-library/src/test/java/org/apache/contrib/joinlibrary/OuterRightJoinTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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 com.google.cloud.dataflow.contrib.joinlibrary;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * This test Outer Right Join functionality.
+ */
+public class OuterRightJoinTest {
+
+  Pipeline p;
+  List<KV<String, Long>> leftListOfKv;
+  List<KV<String, String>> listRightOfKv;
+  List<KV<String, KV<Long, String>>> expectedResult;
+
+  @Before
+  public void setup() {
+
+    p = TestPipeline.create();
+    leftListOfKv = new ArrayList<>();
+    listRightOfKv = new ArrayList<>();
+
+    expectedResult = new ArrayList<>();
+  }
+
+  @Test
+  public void testJoinOneToOneMapping() {
+    leftListOfKv.add(KV.of("Key1", 5L));
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key1", "foo"));
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
+      leftCollection, rightCollection, -1L);
+
+    expectedResult.add(KV.of("Key1", KV.of(5L, "foo")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinOneToManyMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    listRightOfKv.add(KV.of("Key2", "gazonk"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
+      leftCollection, rightCollection, -1L);
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinManyToOneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    leftListOfKv.add(KV.of("Key2", 6L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key2", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
+      leftCollection, rightCollection, -1L);
+
+    expectedResult.add(KV.of("Key2", KV.of(4L, "bar")));
+    expectedResult.add(KV.of("Key2", KV.of(6L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+
+    p.run();
+  }
+
+  @Test
+  public void testJoinNoneToOneMapping() {
+    leftListOfKv.add(KV.of("Key2", 4L));
+    PCollection<KV<String, Long>> leftCollection = p
+        .apply("CreateLeft", Create.of(leftListOfKv));
+
+    listRightOfKv.add(KV.of("Key3", "bar"));
+    PCollection<KV<String, String>> rightCollection = p
+        .apply("CreateRight", Create.of(listRightOfKv));
+
+    PCollection<KV<String, KV<Long, String>>> output = Join.rightOuterJoin(
+      leftCollection, rightCollection, -1L);
+
+    expectedResult.add(KV.of("Key3", KV.of(-1L, "bar")));
+    PAssert.that(output).containsInAnyOrder(expectedResult);
+    p.run();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinLeftCollectionNull() {
+    Join.rightOuterJoin(null, p.apply(Create.of(listRightOfKv)), "");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinRightCollectionNull() {
+    Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), null, -1L);
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testJoinNullValueIsNull() {
+    Join.rightOuterJoin(
+        p.apply("CreateLeft", Create.of(leftListOfKv)),
+        p.apply("CreateRight", Create.of(listRightOfKv)),
+        null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/DebuggingWordCount.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/DebuggingWordCount.java
deleted file mode 100644
index 7134bca..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/DebuggingWordCount.java
+++ /dev/null
@@ -1,199 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.regex.Pattern;
-
-
-/**
- * An example that verifies word counts in Shakespeare and includes Dataflow best practices.
- *
- * <p>This class, {@link DebuggingWordCount}, is the third in a series of four successively more
- * detailed 'word count' examples. You may first want to take a look at {@link MinimalWordCount}
- * and {@link WordCount}. After you've looked at this example, then see the
- * {@link WindowedWordCount} pipeline, for introduction of additional concepts.
- *
- * <p>Basic concepts, also in the MinimalWordCount and WordCount examples:
- * Reading text files; counting a PCollection; executing a Pipeline both locally
- * and using the Dataflow service; defining DoFns.
- *
- * <p>New Concepts:
- * <pre>
- *   1. Logging to Cloud Logging
- *   2. Controlling Dataflow worker log levels
- *   3. Creating a custom aggregator
- *   4. Testing your Pipeline via PAssert
- * </pre>
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- *
- * <p>To execute this pipeline using the Dataflow service and the additional logging discussed
- * below, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
- * }
- * </pre>
- *
- * <p>Note that when you run via <code>mvn exec</code>, you may need to escape
- * the quotations as appropriate for your shell. For example, in <code>bash</code>:
- * <pre>
- * mvn compile exec:java ... \
- *   -Dexec.args="... \
- *     --workerLogLevelOverrides={\\\"com.google.cloud.dataflow.examples\\\":\\\"DEBUG\\\"}"
- * </pre>
- *
- * <p>Concept #2: Dataflow workers which execute user code are configured to log to Cloud
- * Logging by default at "INFO" log level and higher. One may override log levels for specific
- * logging namespaces by specifying:
- * <pre><code>
- *   --workerLogLevelOverrides={"Name1":"Level1","Name2":"Level2",...}
- * </code></pre>
- * For example, by specifying:
- * <pre><code>
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
- * </code></pre>
- * when executing this pipeline using the Dataflow service, Cloud Logging would contain only
- * "DEBUG" or higher level logs for the {@code com.google.cloud.dataflow.examples} package in
- * addition to the default "INFO" or higher level logs. In addition, the default Dataflow worker
- * logging configuration can be overridden by specifying
- * {@code --defaultWorkerLogLevel=<one of TRACE, DEBUG, INFO, WARN, ERROR>}. For example,
- * by specifying {@code --defaultWorkerLogLevel=DEBUG} when executing this pipeline with
- * the Dataflow service, Cloud Logging would contain all "DEBUG" or higher level logs. Note
- * that changing the default worker log level to TRACE or DEBUG will significantly increase
- * the amount of logs output.
- *
- * <p>The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be
- * overridden with {@code --inputFile}.
- */
-public class DebuggingWordCount {
-  /** A DoFn that filters for a specific key based upon a regular expression. */
-  public static class FilterTextFn extends DoFn<KV<String, Long>, KV<String, Long>> {
-    /**
-     * Concept #1: The logger below uses the fully qualified class name of FilterTextFn
-     * as the logger. All log statements emitted by this logger will be referenced by this name
-     * and will be visible in the Cloud Logging UI. Learn more at https://cloud.google.com/logging
-     * about the Cloud Logging UI.
-     */
-    private static final Logger LOG = LoggerFactory.getLogger(FilterTextFn.class);
-
-    private final Pattern filter;
-    public FilterTextFn(String pattern) {
-      filter = Pattern.compile(pattern);
-    }
-
-    /**
-     * Concept #3: A custom aggregator can track values in your pipeline as it runs. Those
-     * values will be displayed in the Dataflow Monitoring UI when this pipeline is run using the
-     * Dataflow service. These aggregators below track the number of matched and unmatched words.
-     * Learn more at https://cloud.google.com/dataflow/pipelines/dataflow-monitoring-intf about
-     * the Dataflow Monitoring UI.
-     */
-    private final Aggregator<Long, Long> matchedWords =
-        createAggregator("matchedWords", new Sum.SumLongFn());
-    private final Aggregator<Long, Long> unmatchedWords =
-        createAggregator("umatchedWords", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      if (filter.matcher(c.element().getKey()).matches()) {
-        // Log at the "DEBUG" level each element that we match. When executing this pipeline
-        // using the Dataflow service, these log lines will appear in the Cloud Logging UI
-        // only if the log level is set to "DEBUG" or lower.
-        LOG.debug("Matched: " + c.element().getKey());
-        matchedWords.addValue(1L);
-        c.output(c.element());
-      } else {
-        // Log at the "TRACE" level each element that is not matched. Different log levels
-        // can be used to control the verbosity of logging providing an effective mechanism
-        // to filter less important information.
-        LOG.trace("Did not match: " + c.element().getKey());
-        unmatchedWords.addValue(1L);
-      }
-    }
-  }
-
-  /**
-   * Options supported by {@link DebuggingWordCount}.
-   *
-   * <p>Inherits standard configuration options and all options defined in
-   * {@link WordCount.WordCountOptions}.
-   */
-  public static interface WordCountOptions extends WordCount.WordCountOptions {
-
-    @Description("Regex filter pattern to use in DebuggingWordCount. "
-        + "Only words matching this pattern will be counted.")
-    @Default.String("Flourish|stomach")
-    String getFilterPattern();
-    void setFilterPattern(String value);
-  }
-
-  public static void main(String[] args) {
-    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-      .as(WordCountOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<KV<String, Long>> filteredWords =
-        p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
-         .apply(new WordCount.CountWords())
-         .apply(ParDo.of(new FilterTextFn(options.getFilterPattern())));
-
-    /**
-     * Concept #4: PAssert is a set of convenient PTransforms in the style of
-     * Hamcrest's collection matchers that can be used when writing Pipeline level tests
-     * to validate the contents of PCollections. PAssert is best used in unit tests
-     * with small data sets but is demonstrated here as a teaching tool.
-     *
-     * <p>Below we verify that the set of filtered words matches our expected counts. Note
-     * that PAssert does not provide any output and that successful completion of the
-     * Pipeline implies that the expectations were met. Learn more at
-     * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline on how to test
-     * your Pipeline and see {@link DebuggingWordCountTest} for an example unit test.
-     */
-    List<KV<String, Long>> expectedResults = Arrays.asList(
-        KV.of("Flourish", 3L),
-        KV.of("stomach", 1L));
-    PAssert.that(filteredWords).containsInAnyOrder(expectedResults);
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCount.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCount.java
deleted file mode 100644
index f3be5a3..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/MinimalWordCount.java
+++ /dev/null
@@ -1,118 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
-import com.google.cloud.dataflow.sdk.values.KV;
-
-
-/**
- * An example that counts words in Shakespeare.
- *
- * <p>This class, {@link MinimalWordCount}, is the first in a series of four successively more
- * detailed 'word count' examples. Here, for simplicity, we don't show any error-checking or
- * argument processing, and focus on construction of the pipeline, which chains together the
- * application of core transforms.
- *
- * <p>Next, see the {@link WordCount} pipeline, then the {@link DebuggingWordCount}, and finally
- * the {@link WindowedWordCount} pipeline, for more detailed examples that introduce additional
- * concepts.
- *
- * <p>Concepts:
- * <pre>
- *   1. Reading data from text files
- *   2. Specifying 'inline' transforms
- *   3. Counting a PCollection
- *   4. Writing data to Cloud Storage as text files
- * </pre>
- *
- * <p>To execute this pipeline, first edit the code to set your project ID, the staging
- * location, and the output location. The specified GCS bucket(s) must already exist.
- *
- * <p>Then, run the pipeline as described in the README. It will be deployed and run using the
- * Dataflow service. No args are required to run the pipeline. You can see the results in your
- * output bucket in the GCS browser.
- */
-public class MinimalWordCount {
-
-  public static void main(String[] args) {
-    // Create a DataflowPipelineOptions object. This object lets us set various execution
-    // options for our pipeline, such as the associated Cloud Platform project and the location
-    // in Google Cloud Storage to stage files.
-    DataflowPipelineOptions options = PipelineOptionsFactory.create()
-      .as(DataflowPipelineOptions.class);
-    options.setRunner(BlockingDataflowPipelineRunner.class);
-    // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud.
-    options.setProject("SET_YOUR_PROJECT_ID_HERE");
-    // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files.
-    options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY");
-
-    // Create the Pipeline object with the options we defined above.
-    Pipeline p = Pipeline.create(options);
-
-    // Apply the pipeline's transforms.
-
-    // Concept #1: Apply a root transform to the pipeline; in this case, TextIO.Read to read a set
-    // of input text files. TextIO.Read returns a PCollection where each element is one line from
-    // the input text (a set of Shakespeare's texts).
-    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
-     // Concept #2: Apply a ParDo transform to our PCollection of text lines. This ParDo invokes a
-     // DoFn (defined in-line) on each element that tokenizes the text line into individual words.
-     // The ParDo returns a PCollection<String>, where each element is an individual word in
-     // Shakespeare's collected texts.
-     .apply(ParDo.named("ExtractWords").of(new DoFn<String, String>() {
-                       @Override
-                       public void processElement(ProcessContext c) {
-                         for (String word : c.element().split("[^a-zA-Z']+")) {
-                           if (!word.isEmpty()) {
-                             c.output(word);
-                           }
-                         }
-                       }
-                     }))
-     // Concept #3: Apply the Count transform to our PCollection of individual words. The Count
-     // transform returns a new PCollection of key/value pairs, where each key represents a unique
-     // word in the text. The associated value is the occurrence count for that word.
-     .apply(Count.<String>perElement())
-     // Apply a MapElements transform that formats our PCollection of word counts into a printable
-     // string, suitable for writing to an output file.
-     .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
-                       @Override
-                       public String apply(KV<String, Long> input) {
-                         return input.getKey() + ": " + input.getValue();
-                       }
-                     }))
-     // Concept #4: Apply a write transform, TextIO.Write, at the end of the pipeline.
-     // TextIO.Write writes the contents of a PCollection (in this case, our PCollection of
-     // formatted strings) to a series of text files in Google Cloud Storage.
-     // CHANGE 3/3: The Google Cloud Storage path is required for outputting the results to.
-     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
-
-    // Run the pipeline.
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java
deleted file mode 100644
index 8f49dd2..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java
+++ /dev/null
@@ -1,270 +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 com.google.cloud.dataflow.examples;
-
-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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-
-/**
- * An example that counts words in text, and can run over either unbounded or bounded input
- * collections.
- *
- * <p>This class, {@link WindowedWordCount}, is the last in a series of four successively more
- * detailed 'word count' examples. First take a look at {@link MinimalWordCount},
- * {@link WordCount}, and {@link DebuggingWordCount}.
- *
- * <p>Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples:
- * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally
- * and using the Dataflow service; defining DoFns; creating a custom aggregator;
- * user-defined PTransforms; defining PipelineOptions.
- *
- * <p>New Concepts:
- * <pre>
- *   1. Unbounded and bounded pipeline input modes
- *   2. Adding timestamps to data
- *   3. PubSub topics as sources
- *   4. Windowing
- *   5. Re-using PTransforms over windowed PCollections
- *   6. Writing to BigQuery
- * </pre>
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- *
- * <p>Optionally specify the input file path via:
- * {@code --inputFile=gs://INPUT_PATH},
- * which defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt}.
- *
- * <p>Specify an output BigQuery dataset and optionally, a table for the output. If you don't
- * specify the table, one will be created for you using the job name. If you don't specify the
- * dataset, a dataset called {@code dataflow-examples} must already exist in your project.
- * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}.
- *
- * <p>Decide whether you want your pipeline to run with 'bounded' (such as files in GCS) or
- * 'unbounded' input (such as a PubSub topic). To run with unbounded input, set
- * {@code --unbounded=true}. Then, optionally specify the Google Cloud PubSub topic to read from
- * via {@code --pubsubTopic=projects/PROJECT_ID/topics/YOUR_TOPIC_NAME}. If the topic does not
- * exist, the pipeline will create one for you. It will delete this topic when it terminates.
- * The pipeline will automatically launch an auxiliary batch pipeline to populate the given PubSub
- * topic with the contents of the {@code --inputFile}, in order to make the example easy to run.
- * If you want to use an independently-populated PubSub topic, indicate this by setting
- * {@code --inputFile=""}. In that case, the auxiliary pipeline will not be started.
- *
- * <p>By default, the pipeline will do fixed windowing, on 1-minute windows.  You can
- * change this interval by setting the {@code --windowSize} parameter, e.g. {@code --windowSize=10}
- * for 10-minute windows.
- */
-public class WindowedWordCount {
-    private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
-    static final int WINDOW_SIZE = 1;  // Default window duration in minutes
-
-  /**
-   * Concept #2: A DoFn that sets the data element timestamp. This is a silly method, just for
-   * this example, for the bounded data case.
-   *
-   * <p>Imagine that many ghosts of Shakespeare are all typing madly at the same time to recreate
-   * his masterworks. Each line of the corpus will get a random associated timestamp somewhere in a
-   * 2-hour period.
-   */
-  static class AddTimestampFn extends DoFn<String, String> {
-    private static final long RAND_RANGE = 7200000; // 2 hours in ms
-
-    @Override
-    public void processElement(ProcessContext c) {
-      // Generate a timestamp that falls somewhere in the past two hours.
-      long randomTimestamp = System.currentTimeMillis()
-        - (int) (Math.random() * RAND_RANGE);
-      /**
-       * Concept #2: Set the data element with that timestamp.
-       */
-      c.outputWithTimestamp(c.element(), new Instant(randomTimestamp));
-    }
-  }
-
-  /** A DoFn that converts a Word and Count into a BigQuery table row. */
-  static class FormatAsTableRowFn extends DoFn<KV<String, Long>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = new TableRow()
-          .set("word", c.element().getKey())
-          .set("count", c.element().getValue())
-          // include a field for the window timestamp
-         .set("window_timestamp", c.timestamp().toString());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Helper method that defines the BigQuery schema used for the output.
-   */
-  private static TableSchema getSchema() {
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("word").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("count").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
-    TableSchema schema = new TableSchema().setFields(fields);
-    return schema;
-  }
-
-  /**
-   * Concept #6: We'll stream the results to a BigQuery table. The BigQuery output source is one
-   * that supports both bounded and unbounded data. This is a helper method that creates a
-   * TableReference from input options, to tell the pipeline where to write its BigQuery results.
-   */
-  private static TableReference getTableReference(Options options) {
-    TableReference tableRef = new TableReference();
-    tableRef.setProjectId(options.getProject());
-    tableRef.setDatasetId(options.getBigQueryDataset());
-    tableRef.setTableId(options.getBigQueryTable());
-    return tableRef;
-  }
-
-  /**
-   * Options supported by {@link WindowedWordCount}.
-   *
-   * <p>Inherits standard example configuration options, which allow specification of the BigQuery
-   * table and the PubSub topic, as well as the {@link WordCount.WordCountOptions} support for
-   * specification of the input file.
-   */
-  public static interface Options extends WordCount.WordCountOptions,
-      DataflowExampleOptions, ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
-    @Description("Fixed window duration, in minutes")
-    @Default.Integer(WINDOW_SIZE)
-    Integer getWindowSize();
-    void setWindowSize(Integer value);
-
-    @Description("Whether to run the pipeline with unbounded input")
-    boolean isUnbounded();
-    void setUnbounded(boolean value);
-  }
-
-  public static void main(String[] args) throws IOException {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    options.setBigQuerySchema(getSchema());
-    // DataflowExampleUtils creates the necessary input sources to simplify execution of this
-    // Pipeline.
-    DataflowExampleUtils exampleDataflowUtils = new DataflowExampleUtils(options,
-      options.isUnbounded());
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    /**
-     * Concept #1: the Dataflow SDK lets us run the same pipeline with either a bounded or
-     * unbounded input source.
-     */
-    PCollection<String> input;
-    if (options.isUnbounded()) {
-      LOG.info("Reading from PubSub.");
-      /**
-       * Concept #3: Read from the PubSub topic. A topic will be created if it wasn't
-       * specified as an argument. The data elements' timestamps will come from the pubsub
-       * injection.
-       */
-      input = pipeline
-          .apply(PubsubIO.Read.topic(options.getPubsubTopic()));
-    } else {
-      /** Else, this is a bounded pipeline. Read from the GCS file. */
-      input = pipeline
-          .apply(TextIO.Read.from(options.getInputFile()))
-          // Concept #2: Add an element timestamp, using an artificial time just to show windowing.
-          // See AddTimestampFn for more detail on this.
-          .apply(ParDo.of(new AddTimestampFn()));
-    }
-
-    /**
-     * Concept #4: Window into fixed windows. The fixed window size for this example defaults to 1
-     * minute (you can change this with a command-line option). See the documentation for more
-     * information on how fixed windows work, and for information on the other types of windowing
-     * available (e.g., sliding windows).
-     */
-    PCollection<String> windowedWords = input
-      .apply(Window.<String>into(
-        FixedWindows.of(Duration.standardMinutes(options.getWindowSize()))));
-
-    /**
-     * Concept #5: Re-use our existing CountWords transform that does not have knowledge of
-     * windows over a PCollection containing windowed values.
-     */
-    PCollection<KV<String, Long>> wordCounts = windowedWords.apply(new WordCount.CountWords());
-
-    /**
-     * Concept #6: Format the results for a BigQuery table, then write to BigQuery.
-     * The BigQuery output source supports both bounded and unbounded data.
-     */
-    wordCounts.apply(ParDo.of(new FormatAsTableRowFn()))
-        .apply(BigQueryIO.Write
-          .to(getTableReference(options))
-          .withSchema(getSchema())
-          .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-          .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND));
-
-    PipelineResult result = pipeline.run();
-
-    /**
-     * To mock unbounded input from PubSub, we'll now start an auxiliary 'injector' pipeline that
-     * runs for a limited time, and publishes to the input PubSub topic.
-     *
-     * With an unbounded input source, you will need to explicitly shut down this pipeline when you
-     * are done with it, so that you do not continue to be charged for the instances. You can do
-     * this via a ctrl-C from the command line, or from the developer's console UI for Dataflow
-     * pipelines. The PubSub topic will also be deleted at this time.
-     */
-    exampleDataflowUtils.mockUnboundedSource(options.getInputFile(), result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/WordCount.java
deleted file mode 100644
index ad08d13..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/WordCount.java
+++ /dev/null
@@ -1,207 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-
-/**
- * An example that counts words in Shakespeare and includes Dataflow best practices.
- *
- * <p>This class, {@link WordCount}, is the second in a series of four successively more detailed
- * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}.
- * After you've looked at this example, then see the {@link DebuggingWordCount}
- * pipeline, for introduction of additional concepts.
- *
- * <p>For a detailed walkthrough of this example, see
- *   <a href="https://cloud.google.com/dataflow/java-sdk/wordcount-example">
- *   https://cloud.google.com/dataflow/java-sdk/wordcount-example
- *   </a>
- *
- * <p>Basic concepts, also in the MinimalWordCount example:
- * Reading text files; counting a PCollection; writing to GCS.
- *
- * <p>New Concepts:
- * <pre>
- *   1. Executing a Pipeline both locally and using the Dataflow service
- *   2. Using ParDo with static DoFns defined out-of-line
- *   3. Building a composite transform
- *   4. Defining your own pipeline options
- * </pre>
- *
- * <p>Concept #1: you can execute this pipeline either locally or using the Dataflow service.
- * These are now command-line options and not hard-coded as they were in the MinimalWordCount
- * example.
- * To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and a local output file or output prefix on GCS:
- * <pre>{@code
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
- *
- * <p>The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be
- * overridden with {@code --inputFile}.
- */
-public class WordCount {
-
-  /**
-   * Concept #2: You can make your pipeline code less verbose by defining your DoFns statically out-
-   * of-line. This DoFn tokenizes lines of text into individual words; we pass it to a ParDo in the
-   * pipeline.
-   */
-  static class ExtractWordsFn extends DoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c) {
-      if (c.element().trim().isEmpty()) {
-        emptyLines.addValue(1L);
-      }
-
-      // Split the line into words.
-      String[] words = c.element().split("[^a-zA-Z']+");
-
-      // Output each word encountered into the output PCollection.
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(word);
-        }
-      }
-    }
-  }
-
-  /** A SimpleFunction that converts a Word and Count into a printable string. */
-  public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
-    @Override
-    public String apply(KV<String, Long> input) {
-      return input.getKey() + ": " + input.getValue();
-    }
-  }
-
-  /**
-   * A PTransform that converts a PCollection containing lines of text into a PCollection of
-   * formatted word counts.
-   *
-   * <p>Concept #3: This is a custom composite transform that bundles two transforms (ParDo and
-   * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse,
-   * modular testing, and an improved monitoring experience.
-   */
-  public static class CountWords extends PTransform<PCollection<String>,
-      PCollection<KV<String, Long>>> {
-    @Override
-    public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<String> words = lines.apply(
-          ParDo.of(new ExtractWordsFn()));
-
-      // Count the number of times each word occurs.
-      PCollection<KV<String, Long>> wordCounts =
-          words.apply(Count.<String>perElement());
-
-      return wordCounts;
-    }
-  }
-
-  /**
-   * Options supported by {@link WordCount}.
-   *
-   * <p>Concept #4: Defining your own configuration options. Here, you can add your own arguments
-   * to be processed by the command-line parser, and specify default values for them. You can then
-   * access the options values in your pipeline code.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  public static interface WordCountOptions extends PipelineOptions {
-    @Description("Path of the file to read from")
-    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
-    String getInputFile();
-    void setInputFile(String value);
-
-    @Description("Path of the file to write to")
-    @Default.InstanceFactory(OutputFactory.class)
-    String getOutput();
-    void setOutput(String value);
-
-    /**
-     * Returns "gs://${YOUR_STAGING_DIRECTORY}/counts.txt" as the default destination.
-     */
-    public static class OutputFactory implements DefaultValueFactory<String> {
-      @Override
-      public String create(PipelineOptions options) {
-        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-        if (dataflowOptions.getStagingLocation() != null) {
-          return GcsPath.fromUri(dataflowOptions.getStagingLocation())
-              .resolve("counts.txt").toString();
-        } else {
-          throw new IllegalArgumentException("Must specify --output or --stagingLocation");
-        }
-      }
-    }
-
-  }
-
-  public static void main(String[] args) {
-    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-      .as(WordCountOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    // 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()))
-     .apply(new CountWords())
-     .apply(MapElements.via(new FormatAsTextFn()))
-     .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleOptions.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleOptions.java
deleted file mode 100644
index 557e903..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/common/DataflowExampleOptions.java
+++ /dev/null
@@ -1,37 +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 com.google.cloud.dataflow.examples.common;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-
-/**
- * Options that can be used to configure the Dataflow examples.
- */
-public interface DataflowExampleOptions extends DataflowPipelineOptions {
-  @Description("Whether to keep jobs running on the Dataflow service after local process exit")
-  @Default.Boolean(false)
-  boolean getKeepJobsRunning();
-  void setKeepJobsRunning(boolean keepJobsRunning);
-
-  @Description("Number of workers to use when executing the injector pipeline")
-  @Default.Integer(1)
-  int getInjectorNumWorkers();
-  void setInjectorNumWorkers(int numWorkers);
-}


[39/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
new file mode 100644
index 0000000..7babc6e
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -0,0 +1,240 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.apache.avro.reflect.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This class is the first in a series of four pipelines that tell a story in a 'gaming' domain.
+ * Concepts: batch processing; reading input from Google Cloud Storage and writing output to
+ * BigQuery; using standalone DoFns; use of the sum by key transform; examples of
+ * Java 8 lambda syntax.
+ *
+ * <p> In this gaming scenario, many users play, as members of different teams, over the course of a
+ * day, and their actions are logged for processing.  Some of the logged game events may be late-
+ * arriving, if users play on mobile devices and go transiently offline for a period.
+ *
+ * <p> This pipeline does batch processing of data collected from gaming events. It calculates the
+ * sum of scores per user, over an entire batch of gaming data (collected, say, for each day). The
+ * batch processing will not include any late data that arrives after the day's cutoff point.
+ *
+ * <p> To execute this pipeline using the Dataflow service and static example input data, specify
+ * the pipeline configuration like this:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ *   --dataset=YOUR-DATASET
+ * }
+ * </pre>
+ * where the BigQuery dataset you specify must already exist.
+ *
+ * <p> Optionally include the --input argument to specify a batch input file.
+ * See the --input default value for example batch data file, or use {@link injector.Injector} to
+ * generate your own batch data.
+  */
+public class UserScore {
+
+  /**
+   * Class to hold info about a game event.
+   */
+  @DefaultCoder(AvroCoder.class)
+  static class GameActionInfo {
+    @Nullable String user;
+    @Nullable String team;
+    @Nullable Integer score;
+    @Nullable Long timestamp;
+
+    public GameActionInfo() {}
+
+    public GameActionInfo(String user, String team, Integer score, Long timestamp) {
+      this.user = user;
+      this.team = team;
+      this.score = score;
+      this.timestamp = timestamp;
+    }
+
+    public String getUser() {
+      return this.user;
+    }
+    public String getTeam() {
+      return this.team;
+    }
+    public Integer getScore() {
+      return this.score;
+    }
+    public String getKey(String keyname) {
+      if (keyname.equals("team")) {
+        return this.team;
+      } else {  // return username as default
+        return this.user;
+      }
+    }
+    public Long getTimestamp() {
+      return this.timestamp;
+    }
+  }
+
+
+  /**
+   * Parses the raw game event info into GameActionInfo objects. Each event line has the following
+   * format: username,teamname,score,timestamp_in_ms,readable_time
+   * e.g.:
+   * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224
+   * The human-readable time string is not used here.
+   */
+  static class ParseEventFn extends DoFn<String, GameActionInfo> {
+
+    // Log and count parse errors.
+    private static final Logger LOG = LoggerFactory.getLogger(ParseEventFn.class);
+    private final Aggregator<Long, Long> numParseErrors =
+        createAggregator("ParseErrors", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      String[] components = c.element().split(",");
+      try {
+        String user = components[0].trim();
+        String team = components[1].trim();
+        Integer score = Integer.parseInt(components[2].trim());
+        Long timestamp = Long.parseLong(components[3].trim());
+        GameActionInfo gInfo = new GameActionInfo(user, team, score, timestamp);
+        c.output(gInfo);
+      } catch (ArrayIndexOutOfBoundsException | NumberFormatException e) {
+        numParseErrors.addValue(1L);
+        LOG.info("Parse error on " + c.element() + ", " + e.getMessage());
+      }
+    }
+  }
+
+  /**
+   * A transform to extract key/score information from GameActionInfo, and sum the scores. The
+   * constructor arg determines whether 'team' or 'user' info is extracted.
+   */
+  // [START DocInclude_USExtractXform]
+  public static class ExtractAndSumScore
+      extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> {
+
+    private final String field;
+
+    ExtractAndSumScore(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public PCollection<KV<String, Integer>> apply(
+        PCollection<GameActionInfo> gameInfo) {
+
+      return gameInfo
+        .apply(MapElements
+            .via((GameActionInfo gInfo) -> KV.of(gInfo.getKey(field), gInfo.getScore()))
+            .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}))
+        .apply(Sum.<String>integersPerKey());
+    }
+  }
+  // [END DocInclude_USExtractXform]
+
+
+  /**
+   * Options supported by {@link UserScore}.
+   */
+  public static interface Options extends PipelineOptions {
+
+    @Description("Path to the data file(s) containing game data.")
+    // The default maps to two large Google Cloud Storage files (each ~12GB) holding two subsequent
+    // day's worth (roughly) of data.
+    @Default.String("gs://dataflow-samples/game/gaming_data*.csv")
+    String getInput();
+    void setInput(String value);
+
+    @Description("BigQuery Dataset to write tables to. Must already exist.")
+    @Validation.Required
+    String getDataset();
+    void setDataset(String value);
+
+    @Description("The BigQuery table name. Should not already exist.")
+    @Default.String("user_score")
+    String getTableName();
+    void setTableName(String value);
+  }
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is passed to the {@link WriteToBigQuery} constructor to write user score sums.
+   */
+  protected static Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>
+    configureBigQueryWrite() {
+    Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
+        new HashMap<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>();
+    tableConfigure.put("user",
+        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("STRING", c -> c.element().getKey()));
+    tableConfigure.put("total_score",
+        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER", c -> c.element().getValue()));
+    return tableConfigure;
+  }
+
+
+  /**
+   * Run a batch pipeline.
+   */
+ // [START DocInclude_USMain]
+  public static void main(String[] args) throws Exception {
+    // Begin constructing a pipeline configured by commandline flags.
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    // Read events from a text file and parse them.
+    pipeline.apply(TextIO.Read.from(options.getInput()))
+      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
+      // Extract and sum username/score pairs from the event data.
+      .apply("ExtractUserScore", new ExtractAndSumScore("user"))
+      .apply("WriteUserScoreSums",
+          new WriteToBigQuery<KV<String, Integer>>(options.getTableName(),
+                                                   configureBigQueryWrite()));
+
+    // Run the batch pipeline.
+    pipeline.run();
+  }
+  // [END DocInclude_USMain]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
new file mode 100644
index 0000000..18ff0a3
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
@@ -0,0 +1,416 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game.injector;
+
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PubsubMessage;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.io.BufferedOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.TimeZone;
+
+
+/**
+ * This is a generator that simulates usage data from a mobile game, and either publishes the data
+ * to a pubsub topic or writes it to a file.
+ *
+ * <p> The general model used by the generator is the following. There is a set of teams with team
+ * members. Each member is scoring points for their team. After some period, a team will dissolve
+ * and a new one will be created in its place. There is also a set of 'Robots', or spammer users.
+ * They hop from team to team. The robots are set to have a higher 'click rate' (generate more
+ * events) than the regular team members.
+ *
+ * <p> Each generated line of data has the following form:
+ * username,teamname,score,timestamp_in_ms,readable_time
+ * e.g.:
+ * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224
+ *
+ * <p> The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if
+ * specified. It takes the following arguments:
+ * {@code Injector project-name (topic-name|none) (filename|none)}.
+ *
+ * <p> To run the Injector in the mode where it publishes to PubSub, you will need to authenticate
+ * locally using project-based service account credentials to avoid running over PubSub
+ * quota.
+ * See https://developers.google.com/identity/protocols/application-default-credentials
+ * for more information on using service account credentials. Set the GOOGLE_APPLICATION_CREDENTIALS
+ * environment variable to point to your downloaded service account credentials before starting the
+ * program, e.g.:
+ * {@code export GOOGLE_APPLICATION_CREDENTIALS=/path/to/your/credentials-key.json}.
+ * If you do not do this, then your injector will only run for a few minutes on your
+ * 'user account' credentials before you will start to see quota error messages like:
+ * "Request throttled due to user QPS limit being reached", and see this exception:
+ * ".com.google.api.client.googleapis.json.GoogleJsonResponseException: 429 Too Many Requests".
+ * Once you've set up your credentials, run the Injector like this":
+  * <pre>{@code
+ * Injector <project-name> <topic-name> none
+ * }
+ * </pre>
+ * The pubsub topic will be created if it does not exist.
+ *
+ * <p> To run the injector in write-to-file-mode, set the topic name to "none" and specify the
+ * filename:
+ * <pre>{@code
+ * Injector <project-name> none <filename>
+ * }
+ * </pre>
+ */
+class Injector {
+  private static Pubsub pubsub;
+  private static Random random = new Random();
+  private static String topic;
+  private static String project;
+  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
+
+  // QPS ranges from 800 to 1000.
+  private static final int MIN_QPS = 800;
+  private static final int QPS_RANGE = 200;
+  // How long to sleep, in ms, between creation of the threads that make API requests to PubSub.
+  private static final int THREAD_SLEEP_MS = 500;
+
+  // Lists used to generate random team names.
+  private static final ArrayList<String> COLORS =
+      new ArrayList<String>(Arrays.asList(
+         "Magenta", "AliceBlue", "Almond", "Amaranth", "Amber",
+         "Amethyst", "AndroidGreen", "AntiqueBrass", "Fuchsia", "Ruby", "AppleGreen",
+         "Apricot", "Aqua", "ArmyGreen", "Asparagus", "Auburn", "Azure", "Banana",
+         "Beige", "Bisque", "BarnRed", "BattleshipGrey"));
+
+  private static final ArrayList<String> ANIMALS =
+      new ArrayList<String>(Arrays.asList(
+         "Echidna", "Koala", "Wombat", "Marmot", "Quokka", "Kangaroo", "Dingo", "Numbat", "Emu",
+         "Wallaby", "CaneToad", "Bilby", "Possum", "Cassowary", "Kookaburra", "Platypus",
+         "Bandicoot", "Cockatoo", "Antechinus"));
+
+  // The list of live teams.
+  private static ArrayList<TeamInfo> liveTeams = new ArrayList<TeamInfo>();
+
+  private static DateTimeFormatter fmt =
+    DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
+        .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
+
+
+  // The total number of robots in the system.
+  private static final int NUM_ROBOTS = 20;
+  // Determines the chance that a team will have a robot team member.
+  private static final int ROBOT_PROBABILITY = 3;
+  private static final int NUM_LIVE_TEAMS = 15;
+  private static final int BASE_MEMBERS_PER_TEAM = 5;
+  private static final int MEMBERS_PER_TEAM = 15;
+  private static final int MAX_SCORE = 20;
+  private static final int LATE_DATA_RATE = 5 * 60 * 2;       // Every 10 minutes
+  private static final int BASE_DELAY_IN_MILLIS = 5 * 60 * 1000;  // 5-10 minute delay
+  private static final int FUZZY_DELAY_IN_MILLIS = 5 * 60 * 1000;
+
+  // The minimum time a 'team' can live.
+  private static final int BASE_TEAM_EXPIRATION_TIME_IN_MINS = 20;
+  private static final int TEAM_EXPIRATION_TIME_IN_MINS = 20;
+
+
+  /**
+   * A class for holding team info: the name of the team, when it started,
+   * and the current team members. Teams may but need not include one robot team member.
+   */
+  private static class TeamInfo {
+    String teamName;
+    long startTimeInMillis;
+    int expirationPeriod;
+    // The team might but need not include 1 robot. Will be non-null if so.
+    String robot;
+    int numMembers;
+
+    private TeamInfo(String teamName, long startTimeInMillis, String robot) {
+      this.teamName = teamName;
+      this.startTimeInMillis = startTimeInMillis;
+      // How long until this team is dissolved.
+      this.expirationPeriod = random.nextInt(TEAM_EXPIRATION_TIME_IN_MINS) +
+        BASE_TEAM_EXPIRATION_TIME_IN_MINS;
+      this.robot = robot;
+      // Determine the number of team members.
+      numMembers = random.nextInt(MEMBERS_PER_TEAM) + BASE_MEMBERS_PER_TEAM;
+    }
+
+    String getTeamName() {
+      return teamName;
+    }
+    String getRobot() {
+      return robot;
+    }
+
+    long getStartTimeInMillis() {
+      return startTimeInMillis;
+    }
+    long getEndTimeInMillis() {
+      return startTimeInMillis + (expirationPeriod * 60 * 1000);
+    }
+    String getRandomUser() {
+      int userNum = random.nextInt(numMembers);
+      return "user" + userNum + "_" + teamName;
+    }
+
+    int numMembers() {
+      return numMembers;
+    }
+
+    @Override
+    public String toString() {
+      return "(" + teamName + ", num members: " + numMembers() + ", starting at: "
+        + startTimeInMillis + ", expires in: " + expirationPeriod + ", robot: " + robot + ")";
+    }
+  }
+
+  /** Utility to grab a random element from an array of Strings. */
+  private static String randomElement(ArrayList<String> list) {
+    int index = random.nextInt(list.size());
+    return list.get(index);
+  }
+
+  /**
+   * Get and return a random team. If the selected team is too old w.r.t its expiration, remove
+   * it, replacing it with a new team.
+   */
+  private static TeamInfo randomTeam(ArrayList<TeamInfo> list) {
+    int index = random.nextInt(list.size());
+    TeamInfo team = list.get(index);
+    // If the selected team is expired, remove it and return a new team.
+    long currTime = System.currentTimeMillis();
+    if ((team.getEndTimeInMillis() < currTime) || team.numMembers() == 0) {
+      System.out.println("\nteam " + team + " is too old; replacing.");
+      System.out.println("start time: " + team.getStartTimeInMillis() +
+        ", end time: " + team.getEndTimeInMillis() +
+        ", current time:" + currTime);
+      removeTeam(index);
+      // Add a new team in its stead.
+      return (addLiveTeam());
+    } else {
+      return team;
+    }
+  }
+
+  /**
+   * Create and add a team. Possibly add a robot to the team.
+   */
+  private static synchronized TeamInfo addLiveTeam() {
+    String teamName = randomElement(COLORS) + randomElement(ANIMALS);
+    String robot = null;
+    // Decide if we want to add a robot to the team.
+    if (random.nextInt(ROBOT_PROBABILITY) == 0) {
+      robot = "Robot-" + random.nextInt(NUM_ROBOTS);
+    }
+    // Create the new team.
+    TeamInfo newTeam = new TeamInfo(teamName, System.currentTimeMillis(), robot);
+    liveTeams.add(newTeam);
+    System.out.println("[+" + newTeam + "]");
+    return newTeam;
+  }
+
+  /**
+   * Remove a specific team.
+   */
+  private static synchronized void removeTeam(int teamIndex) {
+    TeamInfo removedTeam = liveTeams.remove(teamIndex);
+    System.out.println("[-" + removedTeam + "]");
+  }
+
+  /** Generate a user gaming event. */
+  private static String generateEvent(Long currTime, int delayInMillis) {
+    TeamInfo team = randomTeam(liveTeams);
+    String teamName = team.getTeamName();
+    String user;
+    final int parseErrorRate = 900000;
+
+    String robot = team.getRobot();
+    // If the team has an associated robot team member...
+    if (robot != null) {
+      // Then use that robot for the message with some probability.
+      // Set this probability to higher than that used to select any of the 'regular' team
+      // members, so that if there is a robot on the team, it has a higher click rate.
+      if (random.nextInt(team.numMembers() / 2) == 0) {
+        user = robot;
+      } else {
+        user = team.getRandomUser();
+      }
+    } else { // No robot.
+      user = team.getRandomUser();
+    }
+    String event = user + "," + teamName + "," + random.nextInt(MAX_SCORE);
+    // Randomly introduce occasional parse errors. You can see a custom counter tracking the number
+    // of such errors in the Dataflow Monitoring UI, as the example pipeline runs.
+    if (random.nextInt(parseErrorRate) == 0) {
+      System.out.println("Introducing a parse error.");
+      event = "THIS LINE REPRESENTS CORRUPT DATA AND WILL CAUSE A PARSE ERROR";
+    }
+    return addTimeInfoToEvent(event, currTime, delayInMillis);
+  }
+
+  /**
+   * Add time info to a generated gaming event.
+   */
+  private static String addTimeInfoToEvent(String message, Long currTime, int delayInMillis) {
+    String eventTimeString =
+        Long.toString((currTime - delayInMillis) / 1000 * 1000);
+    // Add a (redundant) 'human-readable' date string to make the data semantics more clear.
+    String dateString = fmt.print(currTime);
+    message = message + "," + eventTimeString + "," + dateString;
+    return message;
+  }
+
+  /**
+   * Publish 'numMessages' arbitrary events from live users with the provided delay, to a
+   * PubSub topic.
+   */
+  public static void publishData(int numMessages, int delayInMillis)
+      throws IOException {
+    List<PubsubMessage> pubsubMessages = new ArrayList<>();
+
+    for (int i = 0; i < Math.max(1, numMessages); i++) {
+      Long currTime = System.currentTimeMillis();
+      String message = generateEvent(currTime, delayInMillis);
+      PubsubMessage pubsubMessage = new PubsubMessage()
+              .encodeData(message.getBytes("UTF-8"));
+      pubsubMessage.setAttributes(
+          ImmutableMap.of(TIMESTAMP_ATTRIBUTE,
+              Long.toString((currTime - delayInMillis) / 1000 * 1000)));
+      if (delayInMillis != 0) {
+        System.out.println(pubsubMessage.getAttributes());
+        System.out.println("late data for: " + message);
+      }
+      pubsubMessages.add(pubsubMessage);
+    }
+
+    PublishRequest publishRequest = new PublishRequest();
+    publishRequest.setMessages(pubsubMessages);
+    pubsub.projects().topics().publish(topic, publishRequest).execute();
+  }
+
+  /**
+   * Publish generated events to a file.
+   */
+  public static void publishDataToFile(String fileName, int numMessages, int delayInMillis)
+      throws IOException {
+    PrintWriter out = new PrintWriter(new OutputStreamWriter(
+        new BufferedOutputStream(new FileOutputStream(fileName, true)), "UTF-8"));
+
+    try {
+      for (int i = 0; i < Math.max(1, numMessages); i++) {
+        Long currTime = System.currentTimeMillis();
+        String message = generateEvent(currTime, delayInMillis);
+        out.println(message);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (out != null) {
+        out.flush();
+        out.close();
+      }
+    }
+  }
+
+
+  public static void main(String[] args) throws IOException, InterruptedException {
+    if (args.length < 3) {
+      System.out.println("Usage: Injector project-name (topic-name|none) (filename|none)");
+      System.exit(1);
+    }
+    boolean writeToFile = false;
+    boolean writeToPubsub = true;
+    project = args[0];
+    String topicName = args[1];
+    String fileName = args[2];
+    // The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if
+    // specified; otherwise, it will try to write to a file.
+    if (topicName.equalsIgnoreCase("none")) {
+      writeToFile = true;
+      writeToPubsub = false;
+    }
+    if (writeToPubsub) {
+      // Create the PubSub client.
+      pubsub = InjectorUtils.getClient();
+      // Create the PubSub topic as necessary.
+      topic = InjectorUtils.getFullyQualifiedTopicName(project, topicName);
+      InjectorUtils.createTopic(pubsub, topic);
+      System.out.println("Injecting to topic: " + topic);
+    } else {
+      if (fileName.equalsIgnoreCase("none")) {
+        System.out.println("Filename not specified.");
+        System.exit(1);
+      }
+      System.out.println("Writing to file: " + fileName);
+    }
+    System.out.println("Starting Injector");
+
+    // Start off with some random live teams.
+    while (liveTeams.size() < NUM_LIVE_TEAMS) {
+      addLiveTeam();
+    }
+
+    // Publish messages at a rate determined by the QPS and Thread sleep settings.
+    for (int i = 0; true; i++) {
+      if (Thread.activeCount() > 10) {
+        System.err.println("I'm falling behind!");
+      }
+
+      // Decide if this should be a batch of late data.
+      final int numMessages;
+      final int delayInMillis;
+      if (i % LATE_DATA_RATE == 0) {
+        // Insert delayed data for one user (one message only)
+        delayInMillis = BASE_DELAY_IN_MILLIS + random.nextInt(FUZZY_DELAY_IN_MILLIS);
+        numMessages = 1;
+        System.out.println("DELAY(" + delayInMillis + ", " + numMessages + ")");
+      } else {
+        System.out.print(".");
+        delayInMillis = 0;
+        numMessages = MIN_QPS + random.nextInt(QPS_RANGE);
+      }
+
+      if (writeToFile) { // Won't use threading for the file write.
+        publishDataToFile(fileName, numMessages, delayInMillis);
+      } else { // Write to PubSub.
+        // Start a thread to inject some data.
+        new Thread(){
+          @Override
+          public void run() {
+            try {
+              publishData(numMessages, delayInMillis);
+            } catch (IOException e) {
+              System.err.println(e);
+            }
+          }
+        }.start();
+      }
+
+      // Wait before creating another injector thread.
+      Thread.sleep(THREAD_SLEEP_MS);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..db58650
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java
@@ -0,0 +1,102 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game.injector;
+
+
+import com.google.api.client.googleapis.auth.oauth2.GoogleCredential;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.googleapis.util.Utils;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.client.http.HttpStatusCodes;
+import com.google.api.client.http.HttpTransport;
+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;
+
+class InjectorUtils {
+
+  private static final String APP_NAME = "injector";
+
+  /**
+   * Builds a new Pubsub client and returns it.
+   */
+  public static Pubsub getClient(final HttpTransport httpTransport,
+                                 final JsonFactory jsonFactory)
+           throws IOException {
+      Preconditions.checkNotNull(httpTransport);
+      Preconditions.checkNotNull(jsonFactory);
+      GoogleCredential credential =
+          GoogleCredential.getApplicationDefault(httpTransport, jsonFactory);
+      if (credential.createScopedRequired()) {
+          credential = credential.createScoped(PubsubScopes.all());
+      }
+      if (credential.getClientAuthentication() != null) {
+        System.out.println("\n***Warning! You are not using service account credentials to "
+          + "authenticate.\nYou need to use service account credentials for this example,"
+          + "\nsince user-level credentials do not have enough pubsub quota,\nand so you will run "
+          + "out of PubSub quota very quickly.\nSee "
+          + "https://developers.google.com/identity/protocols/application-default-credentials.");
+        System.exit(1);
+      }
+      HttpRequestInitializer initializer =
+          new RetryHttpInitializerWrapper(credential);
+      return new Pubsub.Builder(httpTransport, jsonFactory, initializer)
+              .setApplicationName(APP_NAME)
+              .build();
+  }
+
+  /**
+   * Builds a new Pubsub client with default HttpTransport and
+   * JsonFactory and returns it.
+   */
+  public static Pubsub getClient() throws IOException {
+      return getClient(Utils.getDefaultTransport(),
+                       Utils.getDefaultJsonFactory());
+  }
+
+
+  /**
+   * Returns the fully qualified topic name for Pub/Sub.
+   */
+  public static String getFullyQualifiedTopicName(
+          final String project, final String topic) {
+      return String.format("projects/%s/topics/%s", project, topic);
+  }
+
+  /**
+   * Create a topic if it doesn't exist.
+   */
+  public static void createTopic(Pubsub client, String fullTopicName)
+      throws IOException {
+    try {
+        client.projects().topics().get(fullTopicName).execute();
+    } catch (GoogleJsonResponseException e) {
+      if (e.getStatusCode() == HttpStatusCodes.STATUS_CODE_NOT_FOUND) {
+        Topic topic = client.projects().topics()
+                .create(fullTopicName, new Topic())
+                .execute();
+        System.out.printf("Topic %s was created.\n", topic.getName());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..9d58837
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java
@@ -0,0 +1,129 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game.injector;
+
+import com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.http.HttpBackOffIOExceptionHandler;
+import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpRequestInitializer;
+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;
+
+/**
+ * RetryHttpInitializerWrapper will automatically retry upon RPC
+ * failures, preserving the auto-refresh behavior of the Google
+ * Credentials.
+ */
+public class RetryHttpInitializerWrapper implements HttpRequestInitializer {
+
+    /**
+     * A private logger.
+     */
+    private static final Logger LOG =
+            Logger.getLogger(RetryHttpInitializerWrapper.class.getName());
+
+    /**
+     * One minutes in miliseconds.
+     */
+    private static final int ONEMINITUES = 60000;
+
+    /**
+     * Intercepts the request for filling in the "Authorization"
+     * header field, as well as recovering from certain unsuccessful
+     * error codes wherein the Credential must refresh its token for a
+     * retry.
+     */
+    private final Credential wrappedCredential;
+
+    /**
+     * A sleeper; you can replace it with a mock in your test.
+     */
+    private final Sleeper sleeper;
+
+    /**
+     * A constructor.
+     *
+     * @param wrappedCredential Credential which will be wrapped and
+     * used for providing auth header.
+     */
+    public RetryHttpInitializerWrapper(final Credential wrappedCredential) {
+        this(wrappedCredential, Sleeper.DEFAULT);
+    }
+
+    /**
+     * A protected constructor only for testing.
+     *
+     * @param wrappedCredential Credential which will be wrapped and
+     * used for providing auth header.
+     * @param sleeper Sleeper for easy testing.
+     */
+    RetryHttpInitializerWrapper(
+            final Credential wrappedCredential, final Sleeper sleeper) {
+        this.wrappedCredential = Preconditions.checkNotNull(wrappedCredential);
+        this.sleeper = sleeper;
+    }
+
+    /**
+     * Initializes the given request.
+     */
+    @Override
+    public final void initialize(final HttpRequest request) {
+        request.setReadTimeout(2 * ONEMINITUES); // 2 minutes read timeout
+        final HttpUnsuccessfulResponseHandler backoffHandler =
+                new HttpBackOffUnsuccessfulResponseHandler(
+                        new ExponentialBackOff())
+                        .setSleeper(sleeper);
+        request.setInterceptor(wrappedCredential);
+        request.setUnsuccessfulResponseHandler(
+                new HttpUnsuccessfulResponseHandler() {
+                    @Override
+                    public boolean handleResponse(
+                            final HttpRequest request,
+                            final HttpResponse response,
+                            final boolean supportsRetry) throws IOException {
+                        if (wrappedCredential.handleResponse(
+                                request, response, supportsRetry)) {
+                            // If credential decides it can handle it,
+                            // the return code or message indicated
+                            // something specific to authentication,
+                            // and no backoff is desired.
+                            return true;
+                        } else if (backoffHandler.handleResponse(
+                                request, response, supportsRetry)) {
+                            // Otherwise, we defer to the judgement of
+                            // our internal backoff handler.
+                            LOG.info("Retrying "
+                                    + request.getUrl().toString());
+                            return true;
+                        } else {
+                            return false;
+                        }
+                    }
+                });
+        request.setIOExceptionHandler(
+                new HttpBackOffIOExceptionHandler(new ExponentialBackOff())
+                        .setSleeper(sleeper));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
new file mode 100644
index 0000000..4bcfb72
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
@@ -0,0 +1,135 @@
+  /*
+ * 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 com.google.cloud.dataflow.examples.complete.game.utils;
+
+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.cloud.dataflow.examples.complete.game.UserScore;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
+import com.google.cloud.dataflow.sdk.options.GcpOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Generate, format, and write BigQuery table row information. Use provided information about
+ * the field names and types, as well as lambda functions that describe how to generate their
+ * values.
+ */
+public class WriteToBigQuery<T>
+    extends PTransform<PCollection<T>, PDone> {
+
+  protected String tableName;
+  protected Map<String, FieldInfo<T>> fieldInfo;
+
+  public WriteToBigQuery() {
+  }
+
+  public WriteToBigQuery(String tableName,
+      Map<String, FieldInfo<T>> fieldInfo) {
+    this.tableName = tableName;
+    this.fieldInfo = fieldInfo;
+  }
+
+  /** Define a class to hold information about output table field definitions. */
+  public static class FieldInfo<T> implements Serializable {
+    // The BigQuery 'type' of the field
+    private String fieldType;
+    // A lambda function to generate the field value
+    private SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fieldFn;
+
+    public FieldInfo(String fieldType,
+        SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fieldFn) {
+      this.fieldType = fieldType;
+      this.fieldFn = fieldFn;
+    }
+
+    String getFieldType() {
+      return this.fieldType;
+    }
+
+    SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> getFieldFn() {
+      return this.fieldFn;
+    }
+  }
+  /** Convert each key/score pair into a BigQuery TableRow as specified by fieldFn. */
+  protected class BuildRowFn extends DoFn<T, TableRow> {
+
+    @Override
+    public void processElement(ProcessContext c) {
+
+      TableRow row = new TableRow();
+      for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
+          String key = entry.getKey();
+          FieldInfo<T> fcnInfo = entry.getValue();
+          SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fcn =
+            fcnInfo.getFieldFn();
+          row.set(key, fcn.apply(c));
+        }
+      c.output(row);
+    }
+  }
+
+  /** Build the output table schema. */
+  protected TableSchema getSchema() {
+    List<TableFieldSchema> fields = new ArrayList<>();
+    for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
+      String key = entry.getKey();
+      FieldInfo<T> fcnInfo = entry.getValue();
+      String bqType = fcnInfo.getFieldType();
+      fields.add(new TableFieldSchema().setName(key).setType(bqType));
+    }
+    return new TableSchema().setFields(fields);
+  }
+
+  @Override
+  public PDone apply(PCollection<T> teamAndScore) {
+    return teamAndScore
+      .apply(ParDo.named("ConvertToRow").of(new BuildRowFn()))
+      .apply(BigQueryIO.Write
+                .to(getTable(teamAndScore.getPipeline(),
+                    tableName))
+                .withSchema(getSchema())
+                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+                .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+  }
+
+  /** Utility to construct an output table reference. */
+  static TableReference getTable(Pipeline pipeline, String tableName) {
+    PipelineOptions options = pipeline.getOptions();
+    TableReference table = new TableReference();
+    table.setDatasetId(options.as(UserScore.Options.class).getDataset());
+    table.setProjectId(options.as(GcpOptions.class).getProject());
+    table.setTableId(tableName);
+    return table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
new file mode 100644
index 0000000..41257ca
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
@@ -0,0 +1,77 @@
+  /*
+ * 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 com.google.cloud.dataflow.examples.complete.game.utils;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+import java.util.Map;
+
+/**
+ * Generate, format, and write BigQuery table row information. Subclasses {@link WriteToBigQuery}
+ * to require windowing; so this subclass may be used for writes that require access to the
+ * context's window information.
+ */
+public class WriteWindowedToBigQuery<T>
+    extends WriteToBigQuery<T> {
+
+  public WriteWindowedToBigQuery(String tableName,
+      Map<String, FieldInfo<T>> fieldInfo) {
+    super(tableName, fieldInfo);
+  }
+
+  /** Convert each key/score pair into a BigQuery TableRow. */
+  protected class BuildRowFn extends DoFn<T, TableRow>
+      implements RequiresWindowAccess {
+
+    @Override
+    public void processElement(ProcessContext c) {
+
+      TableRow row = new TableRow();
+      for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
+          String key = entry.getKey();
+          FieldInfo<T> fcnInfo = entry.getValue();
+          SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fcn =
+            fcnInfo.getFieldFn();
+          row.set(key, fcn.apply(c));
+        }
+      c.output(row);
+    }
+  }
+
+  @Override
+  public PDone apply(PCollection<T> teamAndScore) {
+    return teamAndScore
+      .apply(ParDo.named("ConvertToRow").of(new BuildRowFn()))
+      .apply(BigQueryIO.Write
+                .to(getTable(teamAndScore.getPipeline(),
+                    tableName))
+                .withSchema(getSchema())
+                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+                .withWriteDisposition(WriteDisposition.WRITE_APPEND));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8Test.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8Test.java
deleted file mode 100644
index b34b70d..0000000
--- a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/MinimalWordCountJava8Test.java
+++ /dev/null
@@ -1,104 +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 com.google.cloud.dataflow.examples;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.collect.ImmutableList;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * To keep {@link MinimalWordCountJava8} simple, it is not factored or testable. This test
- * file should be maintained with a copy of its code for a basic smoke test.
- */
-@RunWith(JUnit4.class)
-public class MinimalWordCountJava8Test implements Serializable {
-
-  /**
-   * A basic smoke test that ensures there is no crash at pipeline construction time.
-   */
-  @Test
-  public void testMinimalWordCountJava8() throws Exception {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().as(GcsOptions.class).setGcsUtil(buildMockGcsUtil());
-
-    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
-     .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
-         .withOutputType(new TypeDescriptor<String>() {}))
-     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
-     .apply(Count.<String>perElement())
-     .apply(MapElements
-         .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())
-         .withOutputType(new TypeDescriptor<String>() {}))
-     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
-  }
-
-  private GcsUtil buildMockGcsUtil() throws IOException {
-    GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
-
-    // Any request to open gets a new bogus channel
-    Mockito
-        .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
-        .then(new Answer<SeekableByteChannel>() {
-          @Override
-          public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
-            return FileChannel.open(
-                Files.createTempFile("channel-", ".tmp"),
-                StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
-          }
-        });
-
-    // Any request for expansion returns a list containing the original GcsPath
-    // This is required to pass validation that occurs in TextIO during apply()
-    Mockito
-        .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
-        .then(new Answer<List<GcsPath>>() {
-          @Override
-          public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-            return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-          }
-        });
-
-    return mockGcsUtil;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java
deleted file mode 100644
index 5832c89..0000000
--- a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java
+++ /dev/null
@@ -1,77 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.GameStats.CalculateSpammyUsers;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Tests of GameStats.
- * Because the pipeline was designed for easy readability and explanations, it lacks good
- * modularity for testing. See our testing documentation for better ideas:
- * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline.
- */
-@RunWith(JUnit4.class)
-public class GameStatsTest implements Serializable {
-
-  // User scores
-  static final List<KV<String, Integer>> USER_SCORES = Arrays.asList(
-    KV.of("Robot-2", 66), KV.of("Robot-1", 116), KV.of("user7_AndroidGreenKookaburra", 23),
-    KV.of("user7_AndroidGreenKookaburra", 1),
-    KV.of("user19_BisqueBilby", 14), KV.of("user13_ApricotQuokka", 15),
-    KV.of("user18_BananaEmu", 25), KV.of("user6_AmberEchidna", 8),
-    KV.of("user2_AmberQuokka", 6), KV.of("user0_MagentaKangaroo", 4),
-    KV.of("user0_MagentaKangaroo", 3), KV.of("user2_AmberCockatoo", 13),
-    KV.of("user7_AlmondWallaby", 15), KV.of("user6_AmberNumbat", 11),
-    KV.of("user6_AmberQuokka", 4));
-
-  // The expected list of 'spammers'.
-  static final List<KV<String, Integer>> SPAMMERS = Arrays.asList(
-      KV.of("Robot-2", 66), KV.of("Robot-1", 116));
-
-  /** Test the calculation of 'spammy users'. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testCalculateSpammyUsers() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<KV<String, Integer>> input = p.apply(Create.of(USER_SCORES));
-    PCollection<KV<String, Integer>> output = input.apply(new CalculateSpammyUsers());
-
-    // Check the set of spammers.
-    PAssert.that(output).containsInAnyOrder(SPAMMERS);
-
-    p.run();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java
deleted file mode 100644
index 3fd2c57..0000000
--- a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java
+++ /dev/null
@@ -1,112 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Filter;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Tests of HourlyTeamScore.
- * Because the pipeline was designed for easy readability and explanations, it lacks good
- * modularity for testing. See our testing documentation for better ideas:
- * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline.
- */
-@RunWith(JUnit4.class)
-public class HourlyTeamScoreTest implements Serializable {
-
-  static final String[] GAME_EVENTS_ARRAY = new String[] {
-    "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444",
-    "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444",
-    "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444",
-    "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444",
-    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444",
-    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444",
-    "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444",
-    "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444",
-    // time gap...
-    "user0_AndroidGreenEchidna,AndroidGreenEchidna,0,1447965690000,2015-11-19 12:41:31.053",
-    "user0_MagentaKangaroo,MagentaKangaroo,4,1447965690000,2015-11-19 12:41:31.053",
-    "user2_AmberCockatoo,AmberCockatoo,13,1447965690000,2015-11-19 12:41:31.053",
-    "user18_BananaEmu,BananaEmu,7,1447965690000,2015-11-19 12:41:31.053",
-    "user3_BananaEmu,BananaEmu,17,1447965690000,2015-11-19 12:41:31.053",
-    "user18_BananaEmu,BananaEmu,1,1447965690000,2015-11-19 12:41:31.053",
-    "user18_ApricotCaneToad,ApricotCaneToad,14,1447965690000,2015-11-19 12:41:31.053"
-  };
-
-
-  static final List<String> GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY);
-
-
-  // Used to check the filtering.
-  static final KV[] FILTERED_EVENTS = new KV[] {
-      KV.of("user0_AndroidGreenEchidna", 0), KV.of("user0_MagentaKangaroo", 4),
-      KV.of("user2_AmberCockatoo", 13),
-      KV.of("user18_BananaEmu", 7), KV.of("user3_BananaEmu", 17),
-      KV.of("user18_BananaEmu", 1), KV.of("user18_ApricotCaneToad", 14)
-    };
-
-
-  /** Test the filtering. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testUserScoresFilter() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    final Instant startMinTimestamp = new Instant(1447965680000L);
-
-    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
-
-    PCollection<KV<String, Integer>> output = input
-      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
-
-      .apply("FilterStartTime", Filter.byPredicate(
-          (GameActionInfo gInfo)
-              -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
-      // run a map to access the fields in the result.
-      .apply(MapElements
-          .via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
-          .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
-
-      PAssert.that(output).containsInAnyOrder(FILTERED_EVENTS);
-
-    p.run();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java
deleted file mode 100644
index b907ae7..0000000
--- a/examples/java8/src/test/java/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java
+++ /dev/null
@@ -1,155 +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 com.google.cloud.dataflow.examples.complete.game;
-
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ExtractAndSumScore;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
-import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Tests of UserScore.
- */
-@RunWith(JUnit4.class)
-public class UserScoreTest implements Serializable {
-
-  static final String[] GAME_EVENTS_ARRAY = new String[] {
-    "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444",
-    "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444",
-    "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444",
-    "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444",
-    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444",
-    "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444",
-    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444",
-    "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444",
-    "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444",
-    "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444"
-  };
-
-    static final String[] GAME_EVENTS_ARRAY2 = new String[] {
-    "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444",
-    "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444",
-    "user13_BisqueBilby,BisqueBilby,xxx,1447955630000,2015-11-19 09:53:53.444"
-  };
-
-  static final List<String> GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY);
-  static final List<String> GAME_EVENTS2 = Arrays.asList(GAME_EVENTS_ARRAY2);
-
-  static final List<KV<String, Integer>> USER_SUMS = Arrays.asList(
-      KV.of("user0_MagentaKangaroo", 3), KV.of("user13_ApricotQuokka", 15),
-      KV.of("user6_AmberNumbat", 11), KV.of("user7_AlmondWallaby", 15),
-      KV.of("user7_AndroidGreenKookaburra", 23),
-      KV.of("user19_BisqueBilby", 14));
-
-  static final List<KV<String, Integer>> TEAM_SUMS = Arrays.asList(
-      KV.of("MagentaKangaroo", 3), KV.of("ApricotQuokka", 15),
-      KV.of("AmberNumbat", 11), KV.of("AlmondWallaby", 15),
-      KV.of("AndroidGreenKookaburra", 23),
-      KV.of("BisqueBilby", 14));
-
-  /** Test the ParseEventFn DoFn. */
-  @Test
-  public void testParseEventFn() {
-    DoFnTester<String, GameActionInfo> parseEventFn =
-        DoFnTester.of(new ParseEventFn());
-
-    List<GameActionInfo> results = parseEventFn.processBatch(GAME_EVENTS_ARRAY);
-    Assert.assertEquals(results.size(), 8);
-    Assert.assertEquals(results.get(0).getUser(), "user0_MagentaKangaroo");
-    Assert.assertEquals(results.get(0).getTeam(), "MagentaKangaroo");
-    Assert.assertEquals(results.get(0).getScore(), new Integer(3));
-  }
-
-  /** Tests ExtractAndSumScore("user"). */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testUserScoreSums() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
-
-    PCollection<KV<String, Integer>> output = input
-      .apply(ParDo.of(new ParseEventFn()))
-      // Extract and sum username/score pairs from the event data.
-      .apply("ExtractUserScore", new ExtractAndSumScore("user"));
-
-    // Check the user score sums.
-    PAssert.that(output).containsInAnyOrder(USER_SUMS);
-
-    p.run();
-  }
-
-  /** Tests ExtractAndSumScore("team"). */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testTeamScoreSums() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
-
-    PCollection<KV<String, Integer>> output = input
-      .apply(ParDo.of(new ParseEventFn()))
-      // Extract and sum teamname/score pairs from the event data.
-      .apply("ExtractTeamScore", new ExtractAndSumScore("team"));
-
-    // Check the team score sums.
-    PAssert.that(output).containsInAnyOrder(TEAM_SUMS);
-
-    p.run();
-  }
-
-  /** Test that bad input data is dropped appropriately. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testUserScoresBadInput() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(GAME_EVENTS2).withCoder(StringUtf8Coder.of()));
-
-    PCollection<KV<String, Integer>> extract = input
-      .apply(ParDo.of(new ParseEventFn()))
-      .apply(
-          MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
-          .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
-
-    PAssert.that(extract).empty();
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
new file mode 100644
index 0000000..b34b70d
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
@@ -0,0 +1,104 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.GcsOptions;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * To keep {@link MinimalWordCountJava8} simple, it is not factored or testable. This test
+ * file should be maintained with a copy of its code for a basic smoke test.
+ */
+@RunWith(JUnit4.class)
+public class MinimalWordCountJava8Test implements Serializable {
+
+  /**
+   * A basic smoke test that ensures there is no crash at pipeline construction time.
+   */
+  @Test
+  public void testMinimalWordCountJava8() throws Exception {
+    Pipeline p = TestPipeline.create();
+    p.getOptions().as(GcsOptions.class).setGcsUtil(buildMockGcsUtil());
+
+    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
+     .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
+         .withOutputType(new TypeDescriptor<String>() {}))
+     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
+     .apply(Count.<String>perElement())
+     .apply(MapElements
+         .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())
+         .withOutputType(new TypeDescriptor<String>() {}))
+     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
+  }
+
+  private GcsUtil buildMockGcsUtil() throws IOException {
+    GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
+
+    // Any request to open gets a new bogus channel
+    Mockito
+        .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
+        .then(new Answer<SeekableByteChannel>() {
+          @Override
+          public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+            return FileChannel.open(
+                Files.createTempFile("channel-", ".tmp"),
+                StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+          }
+        });
+
+    // Any request for expansion returns a list containing the original GcsPath
+    // This is required to pass validation that occurs in TextIO during apply()
+    Mockito
+        .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
+        .then(new Answer<List<GcsPath>>() {
+          @Override
+          public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+            return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+          }
+        });
+
+    return mockGcsUtil;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
new file mode 100644
index 0000000..5832c89
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.complete.game.GameStats.CalculateSpammyUsers;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tests of GameStats.
+ * Because the pipeline was designed for easy readability and explanations, it lacks good
+ * modularity for testing. See our testing documentation for better ideas:
+ * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline.
+ */
+@RunWith(JUnit4.class)
+public class GameStatsTest implements Serializable {
+
+  // User scores
+  static final List<KV<String, Integer>> USER_SCORES = Arrays.asList(
+    KV.of("Robot-2", 66), KV.of("Robot-1", 116), KV.of("user7_AndroidGreenKookaburra", 23),
+    KV.of("user7_AndroidGreenKookaburra", 1),
+    KV.of("user19_BisqueBilby", 14), KV.of("user13_ApricotQuokka", 15),
+    KV.of("user18_BananaEmu", 25), KV.of("user6_AmberEchidna", 8),
+    KV.of("user2_AmberQuokka", 6), KV.of("user0_MagentaKangaroo", 4),
+    KV.of("user0_MagentaKangaroo", 3), KV.of("user2_AmberCockatoo", 13),
+    KV.of("user7_AlmondWallaby", 15), KV.of("user6_AmberNumbat", 11),
+    KV.of("user6_AmberQuokka", 4));
+
+  // The expected list of 'spammers'.
+  static final List<KV<String, Integer>> SPAMMERS = Arrays.asList(
+      KV.of("Robot-2", 66), KV.of("Robot-1", 116));
+
+  /** Test the calculation of 'spammy users'. */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testCalculateSpammyUsers() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<KV<String, Integer>> input = p.apply(Create.of(USER_SCORES));
+    PCollection<KV<String, Integer>> output = input.apply(new CalculateSpammyUsers());
+
+    // Check the set of spammers.
+    PAssert.that(output).containsInAnyOrder(SPAMMERS);
+
+    p.run();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
new file mode 100644
index 0000000..3fd2c57
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo;
+import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.testing.PAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tests of HourlyTeamScore.
+ * Because the pipeline was designed for easy readability and explanations, it lacks good
+ * modularity for testing. See our testing documentation for better ideas:
+ * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline.
+ */
+@RunWith(JUnit4.class)
+public class HourlyTeamScoreTest implements Serializable {
+
+  static final String[] GAME_EVENTS_ARRAY = new String[] {
+    "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444",
+    "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444",
+    "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444",
+    "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444",
+    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444",
+    "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444",
+    "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444",
+    "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444",
+    // time gap...
+    "user0_AndroidGreenEchidna,AndroidGreenEchidna,0,1447965690000,2015-11-19 12:41:31.053",
+    "user0_MagentaKangaroo,MagentaKangaroo,4,1447965690000,2015-11-19 12:41:31.053",
+    "user2_AmberCockatoo,AmberCockatoo,13,1447965690000,2015-11-19 12:41:31.053",
+    "user18_BananaEmu,BananaEmu,7,1447965690000,2015-11-19 12:41:31.053",
+    "user3_BananaEmu,BananaEmu,17,1447965690000,2015-11-19 12:41:31.053",
+    "user18_BananaEmu,BananaEmu,1,1447965690000,2015-11-19 12:41:31.053",
+    "user18_ApricotCaneToad,ApricotCaneToad,14,1447965690000,2015-11-19 12:41:31.053"
+  };
+
+
+  static final List<String> GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY);
+
+
+  // Used to check the filtering.
+  static final KV[] FILTERED_EVENTS = new KV[] {
+      KV.of("user0_AndroidGreenEchidna", 0), KV.of("user0_MagentaKangaroo", 4),
+      KV.of("user2_AmberCockatoo", 13),
+      KV.of("user18_BananaEmu", 7), KV.of("user3_BananaEmu", 17),
+      KV.of("user18_BananaEmu", 1), KV.of("user18_ApricotCaneToad", 14)
+    };
+
+
+  /** Test the filtering. */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUserScoresFilter() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    final Instant startMinTimestamp = new Instant(1447965680000L);
+
+    PCollection<String> input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of()));
+
+    PCollection<KV<String, Integer>> output = input
+      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
+
+      .apply("FilterStartTime", Filter.byPredicate(
+          (GameActionInfo gInfo)
+              -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
+      // run a map to access the fields in the result.
+      .apply(MapElements
+          .via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
+          .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
+
+      PAssert.that(output).containsInAnyOrder(FILTERED_EVENTS);
+
+    p.run();
+  }
+
+}


[45/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
new file mode 100644
index 0000000..8f49dd2
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
@@ -0,0 +1,270 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+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.cloud.dataflow.examples.common.DataflowExampleOptions;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * An example that counts words in text, and can run over either unbounded or bounded input
+ * collections.
+ *
+ * <p>This class, {@link WindowedWordCount}, is the last in a series of four successively more
+ * detailed 'word count' examples. First take a look at {@link MinimalWordCount},
+ * {@link WordCount}, and {@link DebuggingWordCount}.
+ *
+ * <p>Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples:
+ * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally
+ * and using the Dataflow service; defining DoFns; creating a custom aggregator;
+ * user-defined PTransforms; defining PipelineOptions.
+ *
+ * <p>New Concepts:
+ * <pre>
+ *   1. Unbounded and bounded pipeline input modes
+ *   2. Adding timestamps to data
+ *   3. PubSub topics as sources
+ *   4. Windowing
+ *   5. Re-using PTransforms over windowed PCollections
+ *   6. Writing to BigQuery
+ * </pre>
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ *
+ * <p>Optionally specify the input file path via:
+ * {@code --inputFile=gs://INPUT_PATH},
+ * which defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt}.
+ *
+ * <p>Specify an output BigQuery dataset and optionally, a table for the output. If you don't
+ * specify the table, one will be created for you using the job name. If you don't specify the
+ * dataset, a dataset called {@code dataflow-examples} must already exist in your project.
+ * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}.
+ *
+ * <p>Decide whether you want your pipeline to run with 'bounded' (such as files in GCS) or
+ * 'unbounded' input (such as a PubSub topic). To run with unbounded input, set
+ * {@code --unbounded=true}. Then, optionally specify the Google Cloud PubSub topic to read from
+ * via {@code --pubsubTopic=projects/PROJECT_ID/topics/YOUR_TOPIC_NAME}. If the topic does not
+ * exist, the pipeline will create one for you. It will delete this topic when it terminates.
+ * The pipeline will automatically launch an auxiliary batch pipeline to populate the given PubSub
+ * topic with the contents of the {@code --inputFile}, in order to make the example easy to run.
+ * If you want to use an independently-populated PubSub topic, indicate this by setting
+ * {@code --inputFile=""}. In that case, the auxiliary pipeline will not be started.
+ *
+ * <p>By default, the pipeline will do fixed windowing, on 1-minute windows.  You can
+ * change this interval by setting the {@code --windowSize} parameter, e.g. {@code --windowSize=10}
+ * for 10-minute windows.
+ */
+public class WindowedWordCount {
+    private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
+    static final int WINDOW_SIZE = 1;  // Default window duration in minutes
+
+  /**
+   * Concept #2: A DoFn that sets the data element timestamp. This is a silly method, just for
+   * this example, for the bounded data case.
+   *
+   * <p>Imagine that many ghosts of Shakespeare are all typing madly at the same time to recreate
+   * his masterworks. Each line of the corpus will get a random associated timestamp somewhere in a
+   * 2-hour period.
+   */
+  static class AddTimestampFn extends DoFn<String, String> {
+    private static final long RAND_RANGE = 7200000; // 2 hours in ms
+
+    @Override
+    public void processElement(ProcessContext c) {
+      // Generate a timestamp that falls somewhere in the past two hours.
+      long randomTimestamp = System.currentTimeMillis()
+        - (int) (Math.random() * RAND_RANGE);
+      /**
+       * Concept #2: Set the data element with that timestamp.
+       */
+      c.outputWithTimestamp(c.element(), new Instant(randomTimestamp));
+    }
+  }
+
+  /** A DoFn that converts a Word and Count into a BigQuery table row. */
+  static class FormatAsTableRowFn extends DoFn<KV<String, Long>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = new TableRow()
+          .set("word", c.element().getKey())
+          .set("count", c.element().getValue())
+          // include a field for the window timestamp
+         .set("window_timestamp", c.timestamp().toString());
+      c.output(row);
+    }
+  }
+
+  /**
+   * Helper method that defines the BigQuery schema used for the output.
+   */
+  private static TableSchema getSchema() {
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("word").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("count").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
+    TableSchema schema = new TableSchema().setFields(fields);
+    return schema;
+  }
+
+  /**
+   * Concept #6: We'll stream the results to a BigQuery table. The BigQuery output source is one
+   * that supports both bounded and unbounded data. This is a helper method that creates a
+   * TableReference from input options, to tell the pipeline where to write its BigQuery results.
+   */
+  private static TableReference getTableReference(Options options) {
+    TableReference tableRef = new TableReference();
+    tableRef.setProjectId(options.getProject());
+    tableRef.setDatasetId(options.getBigQueryDataset());
+    tableRef.setTableId(options.getBigQueryTable());
+    return tableRef;
+  }
+
+  /**
+   * Options supported by {@link WindowedWordCount}.
+   *
+   * <p>Inherits standard example configuration options, which allow specification of the BigQuery
+   * table and the PubSub topic, as well as the {@link WordCount.WordCountOptions} support for
+   * specification of the input file.
+   */
+  public static interface Options extends WordCount.WordCountOptions,
+      DataflowExampleOptions, ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
+    @Description("Fixed window duration, in minutes")
+    @Default.Integer(WINDOW_SIZE)
+    Integer getWindowSize();
+    void setWindowSize(Integer value);
+
+    @Description("Whether to run the pipeline with unbounded input")
+    boolean isUnbounded();
+    void setUnbounded(boolean value);
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    options.setBigQuerySchema(getSchema());
+    // DataflowExampleUtils creates the necessary input sources to simplify execution of this
+    // Pipeline.
+    DataflowExampleUtils exampleDataflowUtils = new DataflowExampleUtils(options,
+      options.isUnbounded());
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    /**
+     * Concept #1: the Dataflow SDK lets us run the same pipeline with either a bounded or
+     * unbounded input source.
+     */
+    PCollection<String> input;
+    if (options.isUnbounded()) {
+      LOG.info("Reading from PubSub.");
+      /**
+       * Concept #3: Read from the PubSub topic. A topic will be created if it wasn't
+       * specified as an argument. The data elements' timestamps will come from the pubsub
+       * injection.
+       */
+      input = pipeline
+          .apply(PubsubIO.Read.topic(options.getPubsubTopic()));
+    } else {
+      /** Else, this is a bounded pipeline. Read from the GCS file. */
+      input = pipeline
+          .apply(TextIO.Read.from(options.getInputFile()))
+          // Concept #2: Add an element timestamp, using an artificial time just to show windowing.
+          // See AddTimestampFn for more detail on this.
+          .apply(ParDo.of(new AddTimestampFn()));
+    }
+
+    /**
+     * Concept #4: Window into fixed windows. The fixed window size for this example defaults to 1
+     * minute (you can change this with a command-line option). See the documentation for more
+     * information on how fixed windows work, and for information on the other types of windowing
+     * available (e.g., sliding windows).
+     */
+    PCollection<String> windowedWords = input
+      .apply(Window.<String>into(
+        FixedWindows.of(Duration.standardMinutes(options.getWindowSize()))));
+
+    /**
+     * Concept #5: Re-use our existing CountWords transform that does not have knowledge of
+     * windows over a PCollection containing windowed values.
+     */
+    PCollection<KV<String, Long>> wordCounts = windowedWords.apply(new WordCount.CountWords());
+
+    /**
+     * Concept #6: Format the results for a BigQuery table, then write to BigQuery.
+     * The BigQuery output source supports both bounded and unbounded data.
+     */
+    wordCounts.apply(ParDo.of(new FormatAsTableRowFn()))
+        .apply(BigQueryIO.Write
+          .to(getTableReference(options))
+          .withSchema(getSchema())
+          .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+          .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND));
+
+    PipelineResult result = pipeline.run();
+
+    /**
+     * To mock unbounded input from PubSub, we'll now start an auxiliary 'injector' pipeline that
+     * runs for a limited time, and publishes to the input PubSub topic.
+     *
+     * With an unbounded input source, you will need to explicitly shut down this pipeline when you
+     * are done with it, so that you do not continue to be charged for the instances. You can do
+     * this via a ctrl-C from the command line, or from the developer's console UI for Dataflow
+     * pipelines. The PubSub topic will also be deleted at this time.
+     */
+    exampleDataflowUtils.mockUnboundedSource(options.getInputFile(), result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..ad08d13
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
@@ -0,0 +1,207 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SimpleFunction;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+
+/**
+ * An example that counts words in Shakespeare and includes Dataflow best practices.
+ *
+ * <p>This class, {@link WordCount}, is the second in a series of four successively more detailed
+ * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}.
+ * After you've looked at this example, then see the {@link DebuggingWordCount}
+ * pipeline, for introduction of additional concepts.
+ *
+ * <p>For a detailed walkthrough of this example, see
+ *   <a href="https://cloud.google.com/dataflow/java-sdk/wordcount-example">
+ *   https://cloud.google.com/dataflow/java-sdk/wordcount-example
+ *   </a>
+ *
+ * <p>Basic concepts, also in the MinimalWordCount example:
+ * Reading text files; counting a PCollection; writing to GCS.
+ *
+ * <p>New Concepts:
+ * <pre>
+ *   1. Executing a Pipeline both locally and using the Dataflow service
+ *   2. Using ParDo with static DoFns defined out-of-line
+ *   3. Building a composite transform
+ *   4. Defining your own pipeline options
+ * </pre>
+ *
+ * <p>Concept #1: you can execute this pipeline either locally or using the Dataflow service.
+ * These are now command-line options and not hard-coded as they were in the MinimalWordCount
+ * example.
+ * To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and a local output file or output prefix on GCS:
+ * <pre>{@code
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and an output prefix on GCS:
+ * <pre>{@code
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ * }</pre>
+ *
+ * <p>The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be
+ * overridden with {@code --inputFile}.
+ */
+public class WordCount {
+
+  /**
+   * Concept #2: You can make your pipeline code less verbose by defining your DoFns statically out-
+   * of-line. This DoFn tokenizes lines of text into individual words; we pass it to a ParDo in the
+   * pipeline.
+   */
+  static class ExtractWordsFn extends DoFn<String, String> {
+    private final Aggregator<Long, Long> emptyLines =
+        createAggregator("emptyLines", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      if (c.element().trim().isEmpty()) {
+        emptyLines.addValue(1L);
+      }
+
+      // Split the line into words.
+      String[] words = c.element().split("[^a-zA-Z']+");
+
+      // Output each word encountered into the output PCollection.
+      for (String word : words) {
+        if (!word.isEmpty()) {
+          c.output(word);
+        }
+      }
+    }
+  }
+
+  /** A SimpleFunction that converts a Word and Count into a printable string. */
+  public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
+    @Override
+    public String apply(KV<String, Long> input) {
+      return input.getKey() + ": " + input.getValue();
+    }
+  }
+
+  /**
+   * A PTransform that converts a PCollection containing lines of text into a PCollection of
+   * formatted word counts.
+   *
+   * <p>Concept #3: This is a custom composite transform that bundles two transforms (ParDo and
+   * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse,
+   * modular testing, and an improved monitoring experience.
+   */
+  public static class CountWords extends PTransform<PCollection<String>,
+      PCollection<KV<String, Long>>> {
+    @Override
+    public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
+
+      // Convert lines of text into individual words.
+      PCollection<String> words = lines.apply(
+          ParDo.of(new ExtractWordsFn()));
+
+      // Count the number of times each word occurs.
+      PCollection<KV<String, Long>> wordCounts =
+          words.apply(Count.<String>perElement());
+
+      return wordCounts;
+    }
+  }
+
+  /**
+   * Options supported by {@link WordCount}.
+   *
+   * <p>Concept #4: Defining your own configuration options. Here, you can add your own arguments
+   * to be processed by the command-line parser, and specify default values for them. You can then
+   * access the options values in your pipeline code.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  public static interface WordCountOptions extends PipelineOptions {
+    @Description("Path of the file to read from")
+    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
+    String getInputFile();
+    void setInputFile(String value);
+
+    @Description("Path of the file to write to")
+    @Default.InstanceFactory(OutputFactory.class)
+    String getOutput();
+    void setOutput(String value);
+
+    /**
+     * Returns "gs://${YOUR_STAGING_DIRECTORY}/counts.txt" as the default destination.
+     */
+    public static class OutputFactory implements DefaultValueFactory<String> {
+      @Override
+      public String create(PipelineOptions options) {
+        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+        if (dataflowOptions.getStagingLocation() != null) {
+          return GcsPath.fromUri(dataflowOptions.getStagingLocation())
+              .resolve("counts.txt").toString();
+        } else {
+          throw new IllegalArgumentException("Must specify --output or --stagingLocation");
+        }
+      }
+    }
+
+  }
+
+  public static void main(String[] args) {
+    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
+      .as(WordCountOptions.class);
+    Pipeline p = Pipeline.create(options);
+
+    // 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()))
+     .apply(new CountWords())
+     .apply(MapElements.via(new FormatAsTextFn()))
+     .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
new file mode 100644
index 0000000..557e903
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
@@ -0,0 +1,37 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+
+/**
+ * Options that can be used to configure the Dataflow examples.
+ */
+public interface DataflowExampleOptions extends DataflowPipelineOptions {
+  @Description("Whether to keep jobs running on the Dataflow service after local process exit")
+  @Default.Boolean(false)
+  boolean getKeepJobsRunning();
+  void setKeepJobsRunning(boolean keepJobsRunning);
+
+  @Description("Number of workers to use when executing the injector pipeline")
+  @Default.Integer(1)
+  int getInjectorNumWorkers();
+  void setInjectorNumWorkers(int numWorkers);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
new file mode 100644
index 0000000..5b98170
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -0,0 +1,488 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.Bigquery.Datasets;
+import com.google.api.services.bigquery.Bigquery.Tables;
+import com.google.api.services.bigquery.model.Dataset;
+import com.google.api.services.bigquery.model.DatasetReference;
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.model.Subscription;
+import com.google.api.services.pubsub.model.Topic;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
+import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
+import com.google.cloud.dataflow.sdk.util.Transport;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * The utility class that sets up and tears down external resources, starts the Google Cloud Pub/Sub
+ * injector, and cancels the streaming and the injector pipelines once the program terminates.
+ *
+ * <p>It is used to run Dataflow examples, such as TrafficMaxLaneFlow and TrafficRoutes.
+ */
+public class DataflowExampleUtils {
+
+  private final DataflowPipelineOptions options;
+  private Bigquery bigQueryClient = null;
+  private Pubsub pubsubClient = null;
+  private Dataflow dataflowClient = null;
+  private Set<DataflowPipelineJob> jobsToCancel = Sets.newHashSet();
+  private List<String> pendingMessages = Lists.newArrayList();
+
+  public DataflowExampleUtils(DataflowPipelineOptions options) {
+    this.options = options;
+  }
+
+  /**
+   * Do resources and runner options setup.
+   */
+  public DataflowExampleUtils(DataflowPipelineOptions options, boolean isUnbounded)
+      throws IOException {
+    this.options = options;
+    setupResourcesAndRunner(isUnbounded);
+  }
+
+  /**
+   * Sets up external resources that are required by the example,
+   * such as Pub/Sub topics and BigQuery tables.
+   *
+   * @throws IOException if there is a problem setting up the resources
+   */
+  public void setup() throws IOException {
+    Sleeper sleeper = Sleeper.DEFAULT;
+    BackOff backOff = new AttemptBoundedExponentialBackOff(3, 200);
+    Throwable lastException = null;
+    try {
+      do {
+        try {
+          setupPubsub();
+          setupBigQueryTable();
+          return;
+        } catch (GoogleJsonResponseException e) {
+          lastException = e;
+        }
+      } while (BackOffUtils.next(sleeper, backOff));
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      // Ignore InterruptedException
+    }
+    Throwables.propagate(lastException);
+  }
+
+  /**
+   * Set up external resources, and configure the runner appropriately.
+   */
+  public void setupResourcesAndRunner(boolean isUnbounded) throws IOException {
+    if (isUnbounded) {
+      options.setStreaming(true);
+    }
+    setup();
+    setupRunner();
+  }
+
+  /**
+   * Sets up the Google Cloud Pub/Sub topic.
+   *
+   * <p>If the topic doesn't exist, a new topic with the given name will be created.
+   *
+   * @throws IOException if there is a problem setting up the Pub/Sub topic
+   */
+  public void setupPubsub() throws IOException {
+    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
+        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
+    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
+      pendingMessages.add("**********************Set Up Pubsub************************");
+      setupPubsubTopic(pubsubOptions.getPubsubTopic());
+      pendingMessages.add("The Pub/Sub topic has been set up for this example: "
+          + pubsubOptions.getPubsubTopic());
+
+      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
+        setupPubsubSubscription(
+            pubsubOptions.getPubsubTopic(), pubsubOptions.getPubsubSubscription());
+        pendingMessages.add("The Pub/Sub subscription has been set up for this example: "
+            + pubsubOptions.getPubsubSubscription());
+      }
+    }
+  }
+
+  /**
+   * Sets up the BigQuery table with the given schema.
+   *
+   * <p>If the table already exists, the schema has to match the given one. Otherwise, the example
+   * will throw a RuntimeException. If the table doesn't exist, a new table with the given schema
+   * will be created.
+   *
+   * @throws IOException if there is a problem setting up the BigQuery table
+   */
+  public void setupBigQueryTable() throws IOException {
+    ExampleBigQueryTableOptions bigQueryTableOptions =
+        options.as(ExampleBigQueryTableOptions.class);
+    if (bigQueryTableOptions.getBigQueryDataset() != null
+        && bigQueryTableOptions.getBigQueryTable() != null
+        && bigQueryTableOptions.getBigQuerySchema() != null) {
+      pendingMessages.add("******************Set Up Big Query Table*******************");
+      setupBigQueryTable(bigQueryTableOptions.getProject(),
+                         bigQueryTableOptions.getBigQueryDataset(),
+                         bigQueryTableOptions.getBigQueryTable(),
+                         bigQueryTableOptions.getBigQuerySchema());
+      pendingMessages.add("The BigQuery table has been set up for this example: "
+          + bigQueryTableOptions.getProject()
+          + ":" + bigQueryTableOptions.getBigQueryDataset()
+          + "." + bigQueryTableOptions.getBigQueryTable());
+    }
+  }
+
+  /**
+   * Tears down external resources that can be deleted upon the example's completion.
+   */
+  private void tearDown() {
+    pendingMessages.add("*************************Tear Down*************************");
+    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
+        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
+    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
+      try {
+        deletePubsubTopic(pubsubOptions.getPubsubTopic());
+        pendingMessages.add("The Pub/Sub topic has been deleted: "
+            + pubsubOptions.getPubsubTopic());
+      } catch (IOException e) {
+        pendingMessages.add("Failed to delete the Pub/Sub topic : "
+            + pubsubOptions.getPubsubTopic());
+      }
+      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
+        try {
+          deletePubsubSubscription(pubsubOptions.getPubsubSubscription());
+          pendingMessages.add("The Pub/Sub subscription has been deleted: "
+              + pubsubOptions.getPubsubSubscription());
+        } catch (IOException e) {
+          pendingMessages.add("Failed to delete the Pub/Sub subscription : "
+              + pubsubOptions.getPubsubSubscription());
+        }
+      }
+    }
+
+    ExampleBigQueryTableOptions bigQueryTableOptions =
+        options.as(ExampleBigQueryTableOptions.class);
+    if (bigQueryTableOptions.getBigQueryDataset() != null
+        && bigQueryTableOptions.getBigQueryTable() != null
+        && bigQueryTableOptions.getBigQuerySchema() != null) {
+      pendingMessages.add("The BigQuery table might contain the example's output, "
+          + "and it is not deleted automatically: "
+          + bigQueryTableOptions.getProject()
+          + ":" + bigQueryTableOptions.getBigQueryDataset()
+          + "." + bigQueryTableOptions.getBigQueryTable());
+      pendingMessages.add("Please go to the Developers Console to delete it manually."
+          + " Otherwise, you may be charged for its usage.");
+    }
+  }
+
+  private void setupBigQueryTable(String projectId, String datasetId, String tableId,
+      TableSchema schema) throws IOException {
+    if (bigQueryClient == null) {
+      bigQueryClient = Transport.newBigQueryClient(options.as(BigQueryOptions.class)).build();
+    }
+
+    Datasets datasetService = bigQueryClient.datasets();
+    if (executeNullIfNotFound(datasetService.get(projectId, datasetId)) == null) {
+      Dataset newDataset = new Dataset().setDatasetReference(
+          new DatasetReference().setProjectId(projectId).setDatasetId(datasetId));
+      datasetService.insert(projectId, newDataset).execute();
+    }
+
+    Tables tableService = bigQueryClient.tables();
+    Table table = executeNullIfNotFound(tableService.get(projectId, datasetId, tableId));
+    if (table == null) {
+      Table newTable = new Table().setSchema(schema).setTableReference(
+          new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId));
+      tableService.insert(projectId, datasetId, newTable).execute();
+    } else if (!table.getSchema().equals(schema)) {
+      throw new RuntimeException(
+          "Table exists and schemas do not match, expecting: " + schema.toPrettyString()
+          + ", actual: " + table.getSchema().toPrettyString());
+    }
+  }
+
+  private void setupPubsubTopic(String topic) throws IOException {
+    if (pubsubClient == null) {
+      pubsubClient = Transport.newPubsubClient(options).build();
+    }
+    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) == null) {
+      pubsubClient.projects().topics().create(topic, new Topic().setName(topic)).execute();
+    }
+  }
+
+  private void setupPubsubSubscription(String topic, String subscription) throws IOException {
+    if (pubsubClient == null) {
+      pubsubClient = Transport.newPubsubClient(options).build();
+    }
+    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) == null) {
+      Subscription subInfo = new Subscription()
+        .setAckDeadlineSeconds(60)
+        .setTopic(topic);
+      pubsubClient.projects().subscriptions().create(subscription, subInfo).execute();
+    }
+  }
+
+  /**
+   * Deletes the Google Cloud Pub/Sub topic.
+   *
+   * @throws IOException if there is a problem deleting the Pub/Sub topic
+   */
+  private void deletePubsubTopic(String topic) throws IOException {
+    if (pubsubClient == null) {
+      pubsubClient = Transport.newPubsubClient(options).build();
+    }
+    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) != null) {
+      pubsubClient.projects().topics().delete(topic).execute();
+    }
+  }
+
+  /**
+   * Deletes the Google Cloud Pub/Sub subscription.
+   *
+   * @throws IOException if there is a problem deleting the Pub/Sub subscription
+   */
+  private void deletePubsubSubscription(String subscription) throws IOException {
+    if (pubsubClient == null) {
+      pubsubClient = Transport.newPubsubClient(options).build();
+    }
+    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) != null) {
+      pubsubClient.projects().subscriptions().delete(subscription).execute();
+    }
+  }
+
+  /**
+   * If this is an unbounded (streaming) pipeline, and both inputFile and pubsub topic are defined,
+   * start an 'injector' pipeline that publishes the contents of the file to the given topic, first
+   * creating the topic if necessary.
+   */
+  public void startInjectorIfNeeded(String inputFile) {
+    ExamplePubsubTopicOptions pubsubTopicOptions = options.as(ExamplePubsubTopicOptions.class);
+    if (pubsubTopicOptions.isStreaming()
+        && !Strings.isNullOrEmpty(inputFile)
+        && !Strings.isNullOrEmpty(pubsubTopicOptions.getPubsubTopic())) {
+      runInjectorPipeline(inputFile, pubsubTopicOptions.getPubsubTopic());
+    }
+  }
+
+  /**
+   * Do some runner setup: check that the DirectPipelineRunner is not used in conjunction with
+   * streaming, and if streaming is specified, use the DataflowPipelineRunner. Return the streaming
+   * flag value.
+   */
+  public void setupRunner() {
+    if (options.isStreaming() && options.getRunner() != DirectPipelineRunner.class) {
+      // In order to cancel the pipelines automatically,
+      // {@literal DataflowPipelineRunner} is forced to be used.
+      options.setRunner(DataflowPipelineRunner.class);
+    }
+  }
+
+  /**
+   * Runs a batch pipeline to inject data into the PubSubIO input topic.
+   *
+   * <p>The injector pipeline will read from the given text file, and inject data
+   * into the Google Cloud Pub/Sub topic.
+   */
+  public void runInjectorPipeline(String inputFile, String topic) {
+    runInjectorPipeline(TextIO.Read.from(inputFile), topic, null);
+  }
+
+  /**
+   * Runs a batch pipeline to inject data into the PubSubIO input topic.
+   *
+   * <p>The injector pipeline will read from the given source, and inject data
+   * into the Google Cloud Pub/Sub topic.
+   */
+  public void runInjectorPipeline(PTransform<? super PBegin, PCollection<String>> readSource,
+                                  String topic,
+                                  String pubsubTimestampTabelKey) {
+    PubsubFileInjector.Bound injector;
+    if (Strings.isNullOrEmpty(pubsubTimestampTabelKey)) {
+      injector = PubsubFileInjector.publish(topic);
+    } else {
+      injector = PubsubFileInjector.withTimestampLabelKey(pubsubTimestampTabelKey).publish(topic);
+    }
+    DataflowPipelineOptions copiedOptions = options.cloneAs(DataflowPipelineOptions.class);
+    if (options.getServiceAccountName() != null) {
+      copiedOptions.setServiceAccountName(options.getServiceAccountName());
+    }
+    if (options.getServiceAccountKeyfile() != null) {
+      copiedOptions.setServiceAccountKeyfile(options.getServiceAccountKeyfile());
+    }
+    copiedOptions.setStreaming(false);
+    copiedOptions.setWorkerHarnessContainerImage(
+        DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
+    copiedOptions.setNumWorkers(options.as(DataflowExampleOptions.class).getInjectorNumWorkers());
+    copiedOptions.setJobName(options.getJobName() + "-injector");
+    Pipeline injectorPipeline = Pipeline.create(copiedOptions);
+    injectorPipeline.apply(readSource)
+                    .apply(IntraBundleParallelization
+                        .of(injector)
+                        .withMaxParallelism(20));
+    PipelineResult result = injectorPipeline.run();
+    if (result instanceof DataflowPipelineJob) {
+      jobsToCancel.add(((DataflowPipelineJob) result));
+    }
+  }
+
+  /**
+   * Runs the provided pipeline to inject data into the PubSubIO input topic.
+   */
+  public void runInjectorPipeline(Pipeline injectorPipeline) {
+    PipelineResult result = injectorPipeline.run();
+    if (result instanceof DataflowPipelineJob) {
+      jobsToCancel.add(((DataflowPipelineJob) result));
+    }
+  }
+
+  /**
+   * Start the auxiliary injector pipeline, then wait for this pipeline to finish.
+   */
+  public void mockUnboundedSource(String inputFile, PipelineResult result) {
+    startInjectorIfNeeded(inputFile);
+    waitToFinish(result);
+  }
+
+  /**
+   * If {@literal DataflowPipelineRunner} or {@literal BlockingDataflowPipelineRunner} is used,
+   * waits for the pipeline to finish and cancels it (and the injector) before the program exists.
+   */
+  public void waitToFinish(PipelineResult result) {
+    if (result instanceof DataflowPipelineJob) {
+      final DataflowPipelineJob job = (DataflowPipelineJob) result;
+      jobsToCancel.add(job);
+      if (!options.as(DataflowExampleOptions.class).getKeepJobsRunning()) {
+        addShutdownHook(jobsToCancel);
+      }
+      try {
+        job.waitToFinish(-1, TimeUnit.SECONDS, new MonitoringUtil.PrintHandler(System.out));
+      } catch (Exception e) {
+        throw new RuntimeException("Failed to wait for job to finish: " + job.getJobId());
+      }
+    } else {
+      // Do nothing if the given PipelineResult doesn't support waitToFinish(),
+      // such as EvaluationResults returned by DirectPipelineRunner.
+      tearDown();
+      printPendingMessages();
+    }
+  }
+
+  private void addShutdownHook(final Collection<DataflowPipelineJob> jobs) {
+    if (dataflowClient == null) {
+      dataflowClient = options.getDataflowClient();
+    }
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        tearDown();
+        printPendingMessages();
+        for (DataflowPipelineJob job : jobs) {
+          System.out.println("Canceling example pipeline: " + job.getJobId());
+          try {
+            job.cancel();
+          } catch (IOException e) {
+            System.out.println("Failed to cancel the job,"
+                + " please go to the Developers Console to cancel it manually");
+            System.out.println(
+                MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId()));
+          }
+        }
+
+        for (DataflowPipelineJob job : jobs) {
+          boolean cancellationVerified = false;
+          for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) {
+            if (job.getState().isTerminal()) {
+              cancellationVerified = true;
+              System.out.println("Canceled example pipeline: " + job.getJobId());
+              break;
+            } else {
+              System.out.println(
+                  "The example pipeline is still running. Verifying the cancellation.");
+            }
+            Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS);
+          }
+          if (!cancellationVerified) {
+            System.out.println("Failed to verify the cancellation for job: " + job.getJobId());
+            System.out.println("Please go to the Developers Console to verify manually:");
+            System.out.println(
+                MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId()));
+          }
+        }
+      }
+    });
+  }
+
+  private void printPendingMessages() {
+    System.out.println();
+    System.out.println("***********************************************************");
+    System.out.println("***********************************************************");
+    for (String message : pendingMessages) {
+      System.out.println(message);
+    }
+    System.out.println("***********************************************************");
+    System.out.println("***********************************************************");
+  }
+
+  private static <T> T executeNullIfNotFound(
+      AbstractGoogleClientRequest<T> request) throws IOException {
+    try {
+      return request.execute();
+    } catch (GoogleJsonResponseException e) {
+      if (e.getStatusCode() == HttpServletResponse.SC_NOT_FOUND) {
+        return null;
+      } else {
+        throw e;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
new file mode 100644
index 0000000..a026f3e
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
@@ -0,0 +1,56 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+
+/**
+ * Options that can be used to configure BigQuery tables in Dataflow examples.
+ * The project defaults to the project being used to run the example.
+ */
+public interface ExampleBigQueryTableOptions extends DataflowPipelineOptions {
+  @Description("BigQuery dataset name")
+  @Default.String("dataflow_examples")
+  String getBigQueryDataset();
+  void setBigQueryDataset(String dataset);
+
+  @Description("BigQuery table name")
+  @Default.InstanceFactory(BigQueryTableFactory.class)
+  String getBigQueryTable();
+  void setBigQueryTable(String table);
+
+  @Description("BigQuery table schema")
+  TableSchema getBigQuerySchema();
+  void setBigQuerySchema(TableSchema schema);
+
+  /**
+   * Returns the job name as the default BigQuery table name.
+   */
+  static class BigQueryTableFactory implements DefaultValueFactory<String> {
+    @Override
+    public String create(PipelineOptions options) {
+      return options.as(DataflowPipelineOptions.class).getJobName()
+          .replace('-', '_');
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
new file mode 100644
index 0000000..bd8cace
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
@@ -0,0 +1,47 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+
+/**
+ * Options that can be used to configure Pub/Sub topic/subscription in Dataflow examples.
+ */
+public interface ExamplePubsubTopicAndSubscriptionOptions extends ExamplePubsubTopicOptions {
+  @Description("Pub/Sub subscription")
+  @Default.InstanceFactory(PubsubSubscriptionFactory.class)
+  String getPubsubSubscription();
+  void setPubsubSubscription(String subscription);
+
+  /**
+   * Returns a default Pub/Sub subscription based on the project and the job names.
+   */
+  static class PubsubSubscriptionFactory implements DefaultValueFactory<String> {
+    @Override
+    public String create(PipelineOptions options) {
+      DataflowPipelineOptions dataflowPipelineOptions =
+          options.as(DataflowPipelineOptions.class);
+      return "projects/" + dataflowPipelineOptions.getProject()
+          + "/subscriptions/" + dataflowPipelineOptions.getJobName();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
new file mode 100644
index 0000000..14df8d9
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
@@ -0,0 +1,47 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+
+/**
+ * Options that can be used to configure Pub/Sub topic in Dataflow examples.
+ */
+public interface ExamplePubsubTopicOptions extends DataflowPipelineOptions {
+  @Description("Pub/Sub topic")
+  @Default.InstanceFactory(PubsubTopicFactory.class)
+  String getPubsubTopic();
+  void setPubsubTopic(String topic);
+
+  /**
+   * Returns a default Pub/Sub topic based on the project and the job names.
+   */
+  static class PubsubTopicFactory implements DefaultValueFactory<String> {
+    @Override
+    public String create(PipelineOptions options) {
+      DataflowPipelineOptions dataflowPipelineOptions =
+          options.as(DataflowPipelineOptions.class);
+      return "projects/" + dataflowPipelineOptions.getProject()
+          + "/topics/" + dataflowPipelineOptions.getJobName();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
new file mode 100644
index 0000000..eb3d1ac
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
@@ -0,0 +1,154 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.common;
+
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PubsubMessage;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
+import com.google.cloud.dataflow.sdk.util.Transport;
+import com.google.common.collect.ImmutableMap;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * A batch Dataflow pipeline for injecting a set of GCS files into
+ * a PubSub topic line by line. Empty lines are skipped.
+ *
+ * <p>This is useful for testing streaming
+ * pipelines. Note that since batch pipelines might retry chunks, this
+ * does _not_ guarantee exactly-once injection of file data. Some lines may
+ * be published multiple times.
+ * </p>
+ */
+public class PubsubFileInjector {
+
+  /**
+   * An incomplete {@code PubsubFileInjector} transform with unbound output topic.
+   */
+  public static class Unbound {
+    private final String timestampLabelKey;
+
+    Unbound() {
+      this.timestampLabelKey = null;
+    }
+
+    Unbound(String timestampLabelKey) {
+      this.timestampLabelKey = timestampLabelKey;
+    }
+
+    Unbound withTimestampLabelKey(String timestampLabelKey) {
+      return new Unbound(timestampLabelKey);
+    }
+
+    public Bound publish(String outputTopic) {
+      return new Bound(outputTopic, timestampLabelKey);
+    }
+  }
+
+  /** A DoFn that publishes non-empty lines to Google Cloud PubSub. */
+  public static class Bound extends DoFn<String, Void> {
+    private final String outputTopic;
+    private final String timestampLabelKey;
+    public transient Pubsub pubsub;
+
+    public Bound(String outputTopic, String timestampLabelKey) {
+      this.outputTopic = outputTopic;
+      this.timestampLabelKey = timestampLabelKey;
+    }
+
+    @Override
+    public void startBundle(Context context) {
+      this.pubsub =
+          Transport.newPubsubClient(context.getPipelineOptions().as(DataflowPipelineOptions.class))
+              .build();
+    }
+
+    @Override
+    public void processElement(ProcessContext c) throws IOException {
+      if (c.element().isEmpty()) {
+        return;
+      }
+      PubsubMessage pubsubMessage = new PubsubMessage();
+      pubsubMessage.encodeData(c.element().getBytes());
+      if (timestampLabelKey != null) {
+        pubsubMessage.setAttributes(
+            ImmutableMap.of(timestampLabelKey, Long.toString(c.timestamp().getMillis())));
+      }
+      PublishRequest publishRequest = new PublishRequest();
+      publishRequest.setMessages(Arrays.asList(pubsubMessage));
+      this.pubsub.projects().topics().publish(outputTopic, publishRequest).execute();
+    }
+  }
+
+  /**
+   * Creates a {@code PubsubFileInjector} transform with the given timestamp label key.
+   */
+  public static Unbound withTimestampLabelKey(String timestampLabelKey) {
+    return new Unbound(timestampLabelKey);
+  }
+
+  /**
+   * Creates a {@code PubsubFileInjector} transform that publishes to the given output topic.
+   */
+  public static Bound publish(String outputTopic) {
+    return new Unbound().publish(outputTopic);
+  }
+
+  /**
+   * Command line parameter options.
+   */
+  private interface PubsubFileInjectorOptions extends PipelineOptions {
+    @Description("GCS location of files.")
+    @Validation.Required
+    String getInput();
+    void setInput(String value);
+
+    @Description("Topic to publish on.")
+    @Validation.Required
+    String getOutputTopic();
+    void setOutputTopic(String value);
+  }
+
+  /**
+   * Sets up and starts streaming pipeline.
+   */
+  public static void main(String[] args) {
+    PubsubFileInjectorOptions options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(PubsubFileInjectorOptions.class);
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    pipeline
+        .apply(TextIO.Read.from(options.getInput()))
+        .apply(IntraBundleParallelization.of(PubsubFileInjector.publish(options.getOutputTopic()))
+            .withMaxParallelism(20));
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..343b17b
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -0,0 +1,517 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete;
+
+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.api.services.datastore.DatastoreV1.Entity;
+import com.google.api.services.datastore.DatastoreV1.Key;
+import com.google.api.services.datastore.DatastoreV1.Value;
+import com.google.api.services.datastore.client.DatastoreHelper;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.DatastoreIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.Partition;
+import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.transforms.Top;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Duration;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * An example that computes the most popular hash tags
+ * for every prefix, which can be used for auto-completion.
+ *
+ * <p>Concepts: Using the same pipeline in both streaming and batch, combiners,
+ *              composite transforms.
+ *
+ * <p>To execute this pipeline using the Dataflow service in batch mode,
+ * specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=DataflowPipelineRunner
+ *   --inputFile=gs://path/to/input*.txt
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service in streaming mode,
+ * specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=DataflowPipelineRunner
+ *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
+ *   --streaming
+ * }</pre>
+ *
+ * <p>This will update the datastore every 10 seconds based on the last
+ * 30 minutes of data received.
+ */
+public class AutoComplete {
+
+  /**
+   * A PTransform that takes as input a list of tokens and returns
+   * the most common tokens per prefix.
+   */
+  public static class ComputeTopCompletions
+      extends PTransform<PCollection<String>, PCollection<KV<String, List<CompletionCandidate>>>> {
+    private final int candidatesPerPrefix;
+    private final boolean recursive;
+
+    protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.recursive = recursive;
+    }
+
+    public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) {
+      return new ComputeTopCompletions(candidatesPerPrefix, recursive);
+    }
+
+    @Override
+    public PCollection<KV<String, List<CompletionCandidate>>> apply(PCollection<String> input) {
+      PCollection<CompletionCandidate> candidates = input
+        // First count how often each token appears.
+        .apply(new Count.PerElement<String>())
+
+        // Map the KV outputs of Count into our own CompletionCandiate class.
+        .apply(ParDo.named("CreateCompletionCandidates").of(
+            new DoFn<KV<String, Long>, CompletionCandidate>() {
+              @Override
+              public void processElement(ProcessContext c) {
+                c.output(new CompletionCandidate(c.element().getKey(), c.element().getValue()));
+              }
+            }));
+
+      // Compute the top via either a flat or recursive algorithm.
+      if (recursive) {
+        return candidates
+          .apply(new ComputeTopRecursive(candidatesPerPrefix, 1))
+          .apply(Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+      } else {
+        return candidates
+          .apply(new ComputeTopFlat(candidatesPerPrefix, 1));
+      }
+    }
+  }
+
+  /**
+   * Lower latency, but more expensive.
+   */
+  private static class ComputeTopFlat
+      extends PTransform<PCollection<CompletionCandidate>,
+                         PCollection<KV<String, List<CompletionCandidate>>>> {
+    private final int candidatesPerPrefix;
+    private final int minPrefix;
+
+    public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.minPrefix = minPrefix;
+    }
+
+    @Override
+    public PCollection<KV<String, List<CompletionCandidate>>> apply(
+        PCollection<CompletionCandidate> input) {
+      return input
+        // For each completion candidate, map it to all prefixes.
+        .apply(ParDo.of(new AllPrefixes(minPrefix)))
+
+        // Find and return the top candiates for each prefix.
+        .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix)
+               .withHotKeyFanout(new HotKeyFanout()));
+    }
+
+    private static class HotKeyFanout implements SerializableFunction<String, Integer> {
+      @Override
+      public Integer apply(String input) {
+        return (int) Math.pow(4, 5 - input.length());
+      }
+    }
+  }
+
+  /**
+   * Cheaper but higher latency.
+   *
+   * <p>Returns two PCollections, the first is top prefixes of size greater
+   * than minPrefix, and the second is top prefixes of size exactly
+   * minPrefix.
+   */
+  private static class ComputeTopRecursive
+      extends PTransform<PCollection<CompletionCandidate>,
+                         PCollectionList<KV<String, List<CompletionCandidate>>>> {
+    private final int candidatesPerPrefix;
+    private final int minPrefix;
+
+    public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.minPrefix = minPrefix;
+    }
+
+    private class KeySizePartitionFn implements PartitionFn<KV<String, List<CompletionCandidate>>> {
+      @Override
+      public int partitionFor(KV<String, List<CompletionCandidate>> elem, int numPartitions) {
+        return elem.getKey().length() > minPrefix ? 0 : 1;
+      }
+    }
+
+    private static class FlattenTops
+        extends DoFn<KV<String, List<CompletionCandidate>>, CompletionCandidate> {
+      @Override
+      public void processElement(ProcessContext c) {
+        for (CompletionCandidate cc : c.element().getValue()) {
+          c.output(cc);
+        }
+      }
+    }
+
+    @Override
+    public PCollectionList<KV<String, List<CompletionCandidate>>> apply(
+          PCollection<CompletionCandidate> input) {
+        if (minPrefix > 10) {
+          // Base case, partitioning to return the output in the expected format.
+          return input
+            .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix))
+            .apply(Partition.of(2, new KeySizePartitionFn()));
+        } else {
+          // If a candidate is in the top N for prefix a...b, it must also be in the top
+          // N for a...bX for every X, which is typlically a much smaller set to consider.
+          // First, compute the top candidate for prefixes of size at least minPrefix + 1.
+          PCollectionList<KV<String, List<CompletionCandidate>>> larger = input
+            .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1));
+          // Consider the top candidates for each prefix of length minPrefix + 1...
+          PCollection<KV<String, List<CompletionCandidate>>> small =
+            PCollectionList
+            .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
+            // ...together with those (previously excluded) candidates of length
+            // exactly minPrefix...
+            .and(input.apply(Filter.byPredicate(
+                new SerializableFunction<CompletionCandidate, Boolean>() {
+                  @Override
+                  public Boolean apply(CompletionCandidate c) {
+                    return c.getValue().length() == minPrefix;
+                  }
+                })))
+            .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
+            // ...set the key to be the minPrefix-length prefix...
+            .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))
+            // ...and (re)apply the Top operator to all of them together.
+            .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix));
+
+          PCollection<KV<String, List<CompletionCandidate>>> flattenLarger = larger
+              .apply("FlattenLarge", Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+
+          return PCollectionList.of(flattenLarger).and(small);
+        }
+    }
+  }
+
+  /**
+   * A DoFn that keys each candidate by all its prefixes.
+   */
+  private static class AllPrefixes
+      extends DoFn<CompletionCandidate, KV<String, CompletionCandidate>> {
+    private final int minPrefix;
+    private final int maxPrefix;
+    public AllPrefixes(int minPrefix) {
+      this(minPrefix, Integer.MAX_VALUE);
+    }
+    public AllPrefixes(int minPrefix, int maxPrefix) {
+      this.minPrefix = minPrefix;
+      this.maxPrefix = maxPrefix;
+    }
+    @Override
+      public void processElement(ProcessContext c) {
+      String word = c.element().value;
+      for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
+        c.output(KV.of(word.substring(0, i), c.element()));
+      }
+    }
+  }
+
+  /**
+   * Class used to store tag-count pairs.
+   */
+  @DefaultCoder(AvroCoder.class)
+  static class CompletionCandidate implements Comparable<CompletionCandidate> {
+    private long count;
+    private String value;
+
+    public CompletionCandidate(String value, long count) {
+      this.value = value;
+      this.count = count;
+    }
+
+    public long getCount() {
+      return count;
+    }
+
+    public String getValue() {
+      return value;
+    }
+
+    // Empty constructor required for Avro decoding.
+    public CompletionCandidate() {}
+
+    @Override
+    public int compareTo(CompletionCandidate o) {
+      if (this.count < o.count) {
+        return -1;
+      } else if (this.count == o.count) {
+        return this.value.compareTo(o.value);
+      } else {
+        return 1;
+      }
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other instanceof CompletionCandidate) {
+        CompletionCandidate that = (CompletionCandidate) other;
+        return this.count == that.count && this.value.equals(that.value);
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return Long.valueOf(count).hashCode() ^ value.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "CompletionCandidate[" + value + ", " + count + "]";
+    }
+  }
+
+  /**
+   * Takes as input a set of strings, and emits each #hashtag found therein.
+   */
+  static class ExtractHashtags extends DoFn<String, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      Matcher m = Pattern.compile("#\\S+").matcher(c.element());
+      while (m.find()) {
+        c.output(m.group().substring(1));
+      }
+    }
+  }
+
+  static class FormatForBigquery extends DoFn<KV<String, List<CompletionCandidate>>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      List<TableRow> completions = new ArrayList<>();
+      for (CompletionCandidate cc : c.element().getValue()) {
+        completions.add(new TableRow()
+          .set("count", cc.getCount())
+          .set("tag", cc.getValue()));
+      }
+      TableRow row = new TableRow()
+        .set("prefix", c.element().getKey())
+        .set("tags", completions);
+      c.output(row);
+    }
+
+    /**
+     * Defines the BigQuery schema used for the output.
+     */
+    static TableSchema getSchema() {
+      List<TableFieldSchema> tagFields = new ArrayList<>();
+      tagFields.add(new TableFieldSchema().setName("count").setType("INTEGER"));
+      tagFields.add(new TableFieldSchema().setName("tag").setType("STRING"));
+      List<TableFieldSchema> fields = new ArrayList<>();
+      fields.add(new TableFieldSchema().setName("prefix").setType("STRING"));
+      fields.add(new TableFieldSchema()
+          .setName("tags").setType("RECORD").setMode("REPEATED").setFields(tagFields));
+      return new TableSchema().setFields(fields);
+    }
+  }
+
+  /**
+   * Takes as input a the top candidates per prefix, and emits an entity
+   * suitable for writing to Datastore.
+   */
+  static class FormatForDatastore extends DoFn<KV<String, List<CompletionCandidate>>, Entity> {
+    private String kind;
+
+    public FormatForDatastore(String kind) {
+      this.kind = kind;
+    }
+
+    @Override
+    public void processElement(ProcessContext c) {
+      Entity.Builder entityBuilder = Entity.newBuilder();
+      Key key = DatastoreHelper.makeKey(kind, c.element().getKey()).build();
+
+      entityBuilder.setKey(key);
+      List<Value> candidates = new ArrayList<>();
+      for (CompletionCandidate tag : c.element().getValue()) {
+        Entity.Builder tagEntity = Entity.newBuilder();
+        tagEntity.addProperty(
+            DatastoreHelper.makeProperty("tag", DatastoreHelper.makeValue(tag.value)));
+        tagEntity.addProperty(
+            DatastoreHelper.makeProperty("count", DatastoreHelper.makeValue(tag.count)));
+        candidates.add(DatastoreHelper.makeValue(tagEntity).setIndexed(false).build());
+      }
+      entityBuilder.addProperty(
+          DatastoreHelper.makeProperty("candidates", DatastoreHelper.makeValue(candidates)));
+      c.output(entityBuilder.build());
+    }
+  }
+
+  /**
+   * Options supported by this class.
+   *
+   * <p>Inherits standard Dataflow configuration options.
+   */
+  private static interface Options extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
+    @Description("Input text file")
+    String getInputFile();
+    void setInputFile(String value);
+
+    @Description("Whether to use the recursive algorithm")
+    @Default.Boolean(true)
+    Boolean getRecursive();
+    void setRecursive(Boolean value);
+
+    @Description("Dataset entity kind")
+    @Default.String("autocomplete-demo")
+    String getKind();
+    void setKind(String value);
+
+    @Description("Whether output to BigQuery")
+    @Default.Boolean(true)
+    Boolean getOutputToBigQuery();
+    void setOutputToBigQuery(Boolean value);
+
+    @Description("Whether output to Datastore")
+    @Default.Boolean(false)
+    Boolean getOutputToDatastore();
+    void setOutputToDatastore(Boolean value);
+
+    @Description("Datastore output dataset ID, defaults to project ID")
+    String getOutputDataset();
+    void setOutputDataset(String value);
+  }
+
+  public static void main(String[] args) throws IOException {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+    if (options.isStreaming()) {
+      // In order to cancel the pipelines automatically,
+      // {@literal DataflowPipelineRunner} is forced to be used.
+      options.setRunner(DataflowPipelineRunner.class);
+    }
+
+    options.setBigQuerySchema(FormatForBigquery.getSchema());
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
+
+    // We support running the same pipeline in either
+    // batch or windowed streaming mode.
+    PTransform<? super PBegin, PCollection<String>> readSource;
+    WindowFn<Object, ?> windowFn;
+    if (options.isStreaming()) {
+      Preconditions.checkArgument(
+          !options.getOutputToDatastore(), "DatastoreIO is not supported in streaming.");
+      dataflowUtils.setupPubsub();
+
+      readSource = PubsubIO.Read.topic(options.getPubsubTopic());
+      windowFn = SlidingWindows.of(Duration.standardMinutes(30)).every(Duration.standardSeconds(5));
+    } else {
+      readSource = TextIO.Read.from(options.getInputFile());
+      windowFn = new GlobalWindows();
+    }
+
+    // Create the pipeline.
+    Pipeline p = Pipeline.create(options);
+    PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
+      .apply(readSource)
+      .apply(ParDo.of(new ExtractHashtags()))
+      .apply(Window.<String>into(windowFn))
+      .apply(ComputeTopCompletions.top(10, options.getRecursive()));
+
+    if (options.getOutputToDatastore()) {
+      toWrite
+      .apply(ParDo.named("FormatForDatastore").of(new FormatForDatastore(options.getKind())))
+      .apply(DatastoreIO.writeTo(MoreObjects.firstNonNull(
+          options.getOutputDataset(), options.getProject())));
+    }
+    if (options.getOutputToBigQuery()) {
+      dataflowUtils.setupBigQueryTable();
+
+      TableReference tableRef = new TableReference();
+      tableRef.setProjectId(options.getProject());
+      tableRef.setDatasetId(options.getBigQueryDataset());
+      tableRef.setTableId(options.getBigQueryTable());
+
+      toWrite
+        .apply(ParDo.of(new FormatForBigquery()))
+        .apply(BigQueryIO.Write
+               .to(tableRef)
+               .withSchema(FormatForBigquery.getSchema())
+               .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+               .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+    }
+
+    // Run the pipeline.
+    PipelineResult result = p.run();
+
+    if (options.isStreaming() && !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.waitToFinish(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
new file mode 100644
index 0000000..5fba154
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
@@ -0,0 +1,44 @@
+
+# "Complete" Examples
+
+This directory contains end-to-end example pipelines that perform complex data processing tasks. They include:
+
+<ul>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java">AutoComplete</a>
+  &mdash; An example that computes the most popular hash tags for every
+  prefix, which can be used for auto-completion. Demonstrates how to use the
+  same pipeline in both streaming and batch, combiners, and composite
+  transforms.</li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java">StreamingWordExtract</a>
+  &mdash; A streaming pipeline example that inputs lines of text from a Cloud
+  Pub/Sub topic, splits each line into individual words, capitalizes those
+  words, and writes the output to a BigQuery table.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java">TfIdf</a>
+  &mdash; An example that computes a basic TF-IDF search table for a directory or
+  Cloud Storage prefix. Demonstrates joining data, side inputs, and logging.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java">TopWikipediaSessions</a>
+  &mdash; An example that reads Wikipedia edit data from Cloud Storage and
+  computes the user with the longest string of edits separated by no more than
+  an hour within each month. Demonstrates using Cloud Dataflow
+  <code>Windowing</code> to perform time-based aggregations of data.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java">TrafficMaxLaneFlow</a>
+  &mdash; A streaming Cloud Dataflow example using BigQuery output in the
+  <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
+  runner, sliding windows, Cloud Pub/Sub topic ingestion, the use of the
+  <code>AvroCoder</code> to encode a custom class, and custom
+  <code>Combine</code> transforms.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java">TrafficRoutes</a>
+  &mdash; A streaming Cloud Dataflow example using BigQuery output in the
+  <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
+  runner, <code>GroupByKey</code>, keyed state, sliding windows, and Cloud
+  Pub/Sub topic ingestion.
+  </li>
+  </ul>
+
+See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
+README](../../../../../../../../../README.md) for
+information about how to run these examples.


[52/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
index 4b429ab..614c72f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.is;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java
index 3a2a492..82a3689 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
index c15c06e..c2c22c0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 
 import org.joda.time.Instant;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
index 3c2d075..31e6b14 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;
 
 import static org.hamcrest.Matchers.arrayWithSize;
 import static org.hamcrest.Matchers.not;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java
index cfac04e..bce0a88 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
+
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM;
 
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java
index eddc8a3..5f75bb8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
-
-import static com.google.cloud.dataflow.sdk.util.Values.asDouble;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.AND;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MIN;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.OR;
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM;
+package org.apache.beam.sdk.util.common;
+
+import static org.apache.beam.sdk.util.Values.asDouble;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.util.common.Counter.CounterMean;
+import org.apache.beam.sdk.util.common.Counter.CounterMean;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTestUtils.java
index 0c4674f..7ed07a1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTestUtils.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTestUtils.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
-import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN;
+import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.util.common.Counter.CounterMean;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.common.Counter.CounterMean;
 
 import org.junit.Assert;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
index dfe6e45..9a5dc92 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java
index 6a0d604..fdd1dfd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.gcsfs;
+package org.apache.beam.sdk.util.gcsfs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java
index 88e1be7..f17f64c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -28,18 +28,18 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 
 import org.joda.time.Instant;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java
index 3e236cb..48d1a30 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
index a26b75b..385ab6c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.coders.Coder;
+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.hamcrest.Matchers;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
index 1212701..df67709 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
@@ -15,22 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.Holder;
-import com.google.cloud.dataflow.sdk.transforms.Max;
-import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn;
-import com.google.cloud.dataflow.sdk.transforms.Min;
-import com.google.cloud.dataflow.sdk.transforms.Min.MinIntegerFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.util.CombineFnUtil;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.transforms.Combine.Holder;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.Max.MaxIntegerFn;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.Min.MinIntegerFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.util.CombineFnUtil;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
index 30d7017..f87e2ae 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.not;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
index c10325e..956bdb1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertThat;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
index 17d86b9..9a8ab30 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.Pipeline;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
index e3d8d68..24896c0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import static com.google.cloud.dataflow.sdk.TestUtils.LINES;
+import static org.apache.beam.sdk.TestUtils.LINES;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import org.junit.Ignore;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TupleTagTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TupleTagTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TupleTagTest.java
index 5ae68cc..e8501b6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TupleTagTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TupleTagTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.startsWith;
@@ -55,12 +55,12 @@ public class TupleTagTest {
 
   @Test
   public void testStaticTupleTag() {
-    assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#0", staticTag.getId());
-    assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#3", staticBlockTag.getId());
-    assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#1", staticMethodTag.getId());
-    assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#2", instanceMethodTag.getId());
+    assertEquals("org.apache.beam.sdk.values.TupleTagTest#0", staticTag.getId());
+    assertEquals("org.apache.beam.sdk.values.TupleTagTest#3", staticBlockTag.getId());
+    assertEquals("org.apache.beam.sdk.values.TupleTagTest#1", staticMethodTag.getId());
+    assertEquals("org.apache.beam.sdk.values.TupleTagTest#2", instanceMethodTag.getId());
     assertEquals(
-        "com.google.cloud.dataflow.sdk.values.TupleTagTest$AnotherClass#0",
+        "org.apache.beam.sdk.values.TupleTagTest$AnotherClass#0",
         AnotherClass.anotherTag.getId());
   }
 
@@ -77,7 +77,7 @@ public class TupleTagTest {
 
     // Check that the name is derived from the method it is created in.
     assertThat(tag.getId().split("#")[0],
-        startsWith("com.google.cloud.dataflow.sdk.values.TupleTagTest.createNonstaticTupleTag"));
+        startsWith("org.apache.beam.sdk.values.TupleTagTest.createNonstaticTupleTag"));
 
     // Check that after the name there is a ':' followed by a line number, and just make
     // sure the line number is big enough to be reasonable, so superficial changes don't break

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
index c1ba500..af48af0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
index f3a570d..0b7018a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java8/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java8/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java b/sdks/java/core/src/test/java8/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
deleted file mode 100644
index 171588c..0000000
--- a/sdks/java/core/src/test/java8/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
+++ /dev/null
@@ -1,92 +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 com.google.cloud.dataflow.sdk.options;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Java 8 tests for {@link PipelineOptionsFactory}.
- */
-@RunWith(JUnit4.class)
-public class PipelineOptionsFactoryJava8Test {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private static interface OptionsWithDefaultMethod extends PipelineOptions {
-    default Number getValue() {
-      return 1024;
-    }
-
-    void setValue(Number value);
-  }
-
-  @Test
-  public void testDefaultMethodIgnoresDefaultImplementation() {
-    OptionsWithDefaultMethod optsWithDefault =
-        PipelineOptionsFactory.as(OptionsWithDefaultMethod.class);
-    assertThat(optsWithDefault.getValue(), nullValue());
-
-    optsWithDefault.setValue(12.25);
-    assertThat(optsWithDefault.getValue(), equalTo(Double.valueOf(12.25)));
-  }
-
-  private static interface ExtendedOptionsWithDefault extends OptionsWithDefaultMethod {}
-
-  @Test
-  public void testDefaultMethodInExtendedClassIgnoresDefaultImplementation() {
-    OptionsWithDefaultMethod extendedOptsWithDefault =
-        PipelineOptionsFactory.as(ExtendedOptionsWithDefault.class);
-    assertThat(extendedOptsWithDefault.getValue(), nullValue());
-
-    extendedOptsWithDefault.setValue(Double.NEGATIVE_INFINITY);
-    assertThat(extendedOptsWithDefault.getValue(), equalTo(Double.NEGATIVE_INFINITY));
-  }
-
-  private static interface Options extends PipelineOptions {
-    Number getValue();
-
-    void setValue(Number value);
-  }
-
-  private static interface SubtypeReturingOptions extends Options {
-    @Override
-    Integer getValue();
-    void setValue(Integer value);
-  }
-
-  @Test
-  public void testReturnTypeConflictThrows() throws Exception {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Method [getValue] has multiple definitions [public abstract java.lang.Integer "
-            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$"
-            + "SubtypeReturingOptions.getValue(), public abstract java.lang.Number "
-            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$Options"
-            + ".getValue()] with different return types for ["
-            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$"
-            + "SubtypeReturingOptions].");
-    PipelineOptionsFactory.as(SubtypeReturingOptions.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
new file mode 100644
index 0000000..ada2a88
--- /dev/null
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryJava8Test.java
@@ -0,0 +1,92 @@
+/*
+ * 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.options;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Java 8 tests for {@link PipelineOptionsFactory}.
+ */
+@RunWith(JUnit4.class)
+public class PipelineOptionsFactoryJava8Test {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private static interface OptionsWithDefaultMethod extends PipelineOptions {
+    default Number getValue() {
+      return 1024;
+    }
+
+    void setValue(Number value);
+  }
+
+  @Test
+  public void testDefaultMethodIgnoresDefaultImplementation() {
+    OptionsWithDefaultMethod optsWithDefault =
+        PipelineOptionsFactory.as(OptionsWithDefaultMethod.class);
+    assertThat(optsWithDefault.getValue(), nullValue());
+
+    optsWithDefault.setValue(12.25);
+    assertThat(optsWithDefault.getValue(), equalTo(Double.valueOf(12.25)));
+  }
+
+  private static interface ExtendedOptionsWithDefault extends OptionsWithDefaultMethod {}
+
+  @Test
+  public void testDefaultMethodInExtendedClassIgnoresDefaultImplementation() {
+    OptionsWithDefaultMethod extendedOptsWithDefault =
+        PipelineOptionsFactory.as(ExtendedOptionsWithDefault.class);
+    assertThat(extendedOptsWithDefault.getValue(), nullValue());
+
+    extendedOptsWithDefault.setValue(Double.NEGATIVE_INFINITY);
+    assertThat(extendedOptsWithDefault.getValue(), equalTo(Double.NEGATIVE_INFINITY));
+  }
+
+  private static interface Options extends PipelineOptions {
+    Number getValue();
+
+    void setValue(Number value);
+  }
+
+  private static interface SubtypeReturingOptions extends Options {
+    @Override
+    Integer getValue();
+    void setValue(Integer value);
+  }
+
+  @Test
+  public void testReturnTypeConflictThrows() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Method [getValue] has multiple definitions [public abstract java.lang.Integer "
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryJava8Test$"
+            + "SubtypeReturingOptions.getValue(), public abstract java.lang.Number "
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryJava8Test$Options"
+            + ".getValue()] with different return types for ["
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryJava8Test$"
+            + "SubtypeReturingOptions].");
+    PipelineOptionsFactory.as(SubtypeReturingOptions.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
index 03709d9..00fc087 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
index 7169468..170071b 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+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.values.PCollection;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
index 0fa8059..5ee10d1 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
index 4f0f232..339e431 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
index 938ea7f..c8283db 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollectionList;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java
index d987395..9318229 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
index a337935..a0d1a63 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
index 1a8a813..fef8d40 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.transforms;
+package org.apache.beam.sdk.transforms;
 
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+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.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Instant;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 bd63b75..43c990a 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
@@ -18,18 +18,18 @@
 package ${package};
 
 import ${package}.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+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;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,7 +71,7 @@ import java.util.regex.Pattern;
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
  * </pre>
  *
@@ -80,7 +80,7 @@ import java.util.regex.Pattern;
  * <pre>
  * mvn compile exec:java ... \
  *   -Dexec.args="... \
- *     --workerLogLevelOverrides={\\\"com.google.cloud.dataflow.examples\\\":\\\"DEBUG\\\"}"
+ *     --workerLogLevelOverrides={\\\"org.apache.beam.examples\\\":\\\"DEBUG\\\"}"
  * </pre>
  *
  * <p>Concept #2: Dataflow workers which execute user code are configured to log to Cloud
@@ -91,10 +91,10 @@ import java.util.regex.Pattern;
  * </code></pre>
  * For example, by specifying:
  * <pre><code>
- *   --workerLogLevelOverrides={"com.google.cloud.dataflow.examples":"DEBUG"}
+ *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * </code></pre>
  * when executing this pipeline using the Dataflow service, Cloud Logging would contain only
- * "DEBUG" or higher level logs for the {@code com.google.cloud.dataflow.examples} package in
+ * "DEBUG" or higher level logs for the {@code org.apache.beam.examples} package in
  * addition to the default "INFO" or higher level logs. In addition, the default Dataflow worker
  * logging configuration can be overridden by specifying
  * {@code --defaultWorkerLogLevel=<one of TRACE, DEBUG, INFO, WARN, ERROR>}. For example,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
index 8b296fa..4b2bb7b 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
@@ -17,15 +17,15 @@
  */
 package ${package};
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
index 5e8e80b..f997521 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
@@ -22,20 +22,20 @@ 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 ${package}.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+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;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 58cc04c..8c89283 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
@@ -17,23 +17,23 @@
  */
 package ${package};
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+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.Count;
+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.Sum;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
index 1c140df..9e7136d 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
@@ -17,9 +17,9 @@
  */
 package ${package}.common;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
 
 /**
  * Options that can be used to configure the Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
index 5042c2e..0ad449a 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
@@ -30,17 +30,17 @@ import com.google.api.services.bigquery.model.TableSchema;
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.model.Topic;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.Transport;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.runners.DataflowPipelineJob;
+import org.apache.beam.sdk.runners.DataflowPipelineRunner;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.IntraBundleParallelization;
+import org.apache.beam.sdk.util.MonitoringUtil;
+import org.apache.beam.sdk.util.Transport;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
index dba8b51..9e65c4f 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
@@ -18,11 +18,11 @@
 package ${package}.common;
 
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * Options that can be used to configure BigQuery tables in Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
index 30b9e1a..b5acf69 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
@@ -17,11 +17,11 @@
  */
 package ${package}.common;
 
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 /**
  * Options that can be used to configure Pub/Sub topic in Dataflow examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
index 096d9d2..d8bab1f 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
@@ -20,16 +20,16 @@ package ${package}.common;
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.model.PublishRequest;
 import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
-import com.google.cloud.dataflow.sdk.util.Transport;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.IntraBundleParallelization;
+import org.apache.beam.sdk.util.Transport;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
index b3319b6..b921875 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
@@ -20,15 +20,15 @@ package ${package};
 import ${package}.WordCount.CountWords;
 import ${package}.WordCount.ExtractWordsFn;
 import ${package}.WordCount.FormatAsTextFn;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
index 7542dbc..2146b77 100644
--- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
@@ -17,11 +17,11 @@
  */
 package ${package};
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
index 1168d36..6cd27e7 100644
--- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
@@ -17,11 +17,11 @@
  */
 package it.pkg;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/travis/test_wordcount.sh
----------------------------------------------------------------------
diff --git a/travis/test_wordcount.sh b/travis/test_wordcount.sh
index fdd878d..89657a9 100755
--- a/travis/test_wordcount.sh
+++ b/travis/test_wordcount.sh
@@ -53,7 +53,7 @@ function run_via_mvn {
 
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='mvn exec:java -f pom.xml -pl examples/java \
-    -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \
+    -Dexec.mainClass=org.apache.beam.examples.WordCount \
     -Dexec.args="--runner=DirectPipelineRunner --inputFile='"$input"' --output='"$outfile_prefix"'"'
   echo "$name: Running $cmd" >&2
   sh -c "$cmd"
@@ -67,7 +67,7 @@ function run_bundled {
 
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='java -cp '"$JAR_FILE"' \
-    com.google.cloud.dataflow.examples.WordCount \
+    org.apache.beam.examples.WordCount \
     --runner=DirectPipelineRunner \
     --inputFile='"'$input'"' \
     --output='"$outfile_prefix"


[40/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java
deleted file mode 100644
index 18ff0a3..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java
+++ /dev/null
@@ -1,416 +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 com.google.cloud.dataflow.examples.complete.game.injector;
-
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.DateTimeZone;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-import java.io.BufferedOutputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-import java.util.TimeZone;
-
-
-/**
- * This is a generator that simulates usage data from a mobile game, and either publishes the data
- * to a pubsub topic or writes it to a file.
- *
- * <p> The general model used by the generator is the following. There is a set of teams with team
- * members. Each member is scoring points for their team. After some period, a team will dissolve
- * and a new one will be created in its place. There is also a set of 'Robots', or spammer users.
- * They hop from team to team. The robots are set to have a higher 'click rate' (generate more
- * events) than the regular team members.
- *
- * <p> Each generated line of data has the following form:
- * username,teamname,score,timestamp_in_ms,readable_time
- * e.g.:
- * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224
- *
- * <p> The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if
- * specified. It takes the following arguments:
- * {@code Injector project-name (topic-name|none) (filename|none)}.
- *
- * <p> To run the Injector in the mode where it publishes to PubSub, you will need to authenticate
- * locally using project-based service account credentials to avoid running over PubSub
- * quota.
- * See https://developers.google.com/identity/protocols/application-default-credentials
- * for more information on using service account credentials. Set the GOOGLE_APPLICATION_CREDENTIALS
- * environment variable to point to your downloaded service account credentials before starting the
- * program, e.g.:
- * {@code export GOOGLE_APPLICATION_CREDENTIALS=/path/to/your/credentials-key.json}.
- * If you do not do this, then your injector will only run for a few minutes on your
- * 'user account' credentials before you will start to see quota error messages like:
- * "Request throttled due to user QPS limit being reached", and see this exception:
- * ".com.google.api.client.googleapis.json.GoogleJsonResponseException: 429 Too Many Requests".
- * Once you've set up your credentials, run the Injector like this":
-  * <pre>{@code
- * Injector <project-name> <topic-name> none
- * }
- * </pre>
- * The pubsub topic will be created if it does not exist.
- *
- * <p> To run the injector in write-to-file-mode, set the topic name to "none" and specify the
- * filename:
- * <pre>{@code
- * Injector <project-name> none <filename>
- * }
- * </pre>
- */
-class Injector {
-  private static Pubsub pubsub;
-  private static Random random = new Random();
-  private static String topic;
-  private static String project;
-  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
-
-  // QPS ranges from 800 to 1000.
-  private static final int MIN_QPS = 800;
-  private static final int QPS_RANGE = 200;
-  // How long to sleep, in ms, between creation of the threads that make API requests to PubSub.
-  private static final int THREAD_SLEEP_MS = 500;
-
-  // Lists used to generate random team names.
-  private static final ArrayList<String> COLORS =
-      new ArrayList<String>(Arrays.asList(
-         "Magenta", "AliceBlue", "Almond", "Amaranth", "Amber",
-         "Amethyst", "AndroidGreen", "AntiqueBrass", "Fuchsia", "Ruby", "AppleGreen",
-         "Apricot", "Aqua", "ArmyGreen", "Asparagus", "Auburn", "Azure", "Banana",
-         "Beige", "Bisque", "BarnRed", "BattleshipGrey"));
-
-  private static final ArrayList<String> ANIMALS =
-      new ArrayList<String>(Arrays.asList(
-         "Echidna", "Koala", "Wombat", "Marmot", "Quokka", "Kangaroo", "Dingo", "Numbat", "Emu",
-         "Wallaby", "CaneToad", "Bilby", "Possum", "Cassowary", "Kookaburra", "Platypus",
-         "Bandicoot", "Cockatoo", "Antechinus"));
-
-  // The list of live teams.
-  private static ArrayList<TeamInfo> liveTeams = new ArrayList<TeamInfo>();
-
-  private static DateTimeFormatter fmt =
-    DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
-        .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
-
-
-  // The total number of robots in the system.
-  private static final int NUM_ROBOTS = 20;
-  // Determines the chance that a team will have a robot team member.
-  private static final int ROBOT_PROBABILITY = 3;
-  private static final int NUM_LIVE_TEAMS = 15;
-  private static final int BASE_MEMBERS_PER_TEAM = 5;
-  private static final int MEMBERS_PER_TEAM = 15;
-  private static final int MAX_SCORE = 20;
-  private static final int LATE_DATA_RATE = 5 * 60 * 2;       // Every 10 minutes
-  private static final int BASE_DELAY_IN_MILLIS = 5 * 60 * 1000;  // 5-10 minute delay
-  private static final int FUZZY_DELAY_IN_MILLIS = 5 * 60 * 1000;
-
-  // The minimum time a 'team' can live.
-  private static final int BASE_TEAM_EXPIRATION_TIME_IN_MINS = 20;
-  private static final int TEAM_EXPIRATION_TIME_IN_MINS = 20;
-
-
-  /**
-   * A class for holding team info: the name of the team, when it started,
-   * and the current team members. Teams may but need not include one robot team member.
-   */
-  private static class TeamInfo {
-    String teamName;
-    long startTimeInMillis;
-    int expirationPeriod;
-    // The team might but need not include 1 robot. Will be non-null if so.
-    String robot;
-    int numMembers;
-
-    private TeamInfo(String teamName, long startTimeInMillis, String robot) {
-      this.teamName = teamName;
-      this.startTimeInMillis = startTimeInMillis;
-      // How long until this team is dissolved.
-      this.expirationPeriod = random.nextInt(TEAM_EXPIRATION_TIME_IN_MINS) +
-        BASE_TEAM_EXPIRATION_TIME_IN_MINS;
-      this.robot = robot;
-      // Determine the number of team members.
-      numMembers = random.nextInt(MEMBERS_PER_TEAM) + BASE_MEMBERS_PER_TEAM;
-    }
-
-    String getTeamName() {
-      return teamName;
-    }
-    String getRobot() {
-      return robot;
-    }
-
-    long getStartTimeInMillis() {
-      return startTimeInMillis;
-    }
-    long getEndTimeInMillis() {
-      return startTimeInMillis + (expirationPeriod * 60 * 1000);
-    }
-    String getRandomUser() {
-      int userNum = random.nextInt(numMembers);
-      return "user" + userNum + "_" + teamName;
-    }
-
-    int numMembers() {
-      return numMembers;
-    }
-
-    @Override
-    public String toString() {
-      return "(" + teamName + ", num members: " + numMembers() + ", starting at: "
-        + startTimeInMillis + ", expires in: " + expirationPeriod + ", robot: " + robot + ")";
-    }
-  }
-
-  /** Utility to grab a random element from an array of Strings. */
-  private static String randomElement(ArrayList<String> list) {
-    int index = random.nextInt(list.size());
-    return list.get(index);
-  }
-
-  /**
-   * Get and return a random team. If the selected team is too old w.r.t its expiration, remove
-   * it, replacing it with a new team.
-   */
-  private static TeamInfo randomTeam(ArrayList<TeamInfo> list) {
-    int index = random.nextInt(list.size());
-    TeamInfo team = list.get(index);
-    // If the selected team is expired, remove it and return a new team.
-    long currTime = System.currentTimeMillis();
-    if ((team.getEndTimeInMillis() < currTime) || team.numMembers() == 0) {
-      System.out.println("\nteam " + team + " is too old; replacing.");
-      System.out.println("start time: " + team.getStartTimeInMillis() +
-        ", end time: " + team.getEndTimeInMillis() +
-        ", current time:" + currTime);
-      removeTeam(index);
-      // Add a new team in its stead.
-      return (addLiveTeam());
-    } else {
-      return team;
-    }
-  }
-
-  /**
-   * Create and add a team. Possibly add a robot to the team.
-   */
-  private static synchronized TeamInfo addLiveTeam() {
-    String teamName = randomElement(COLORS) + randomElement(ANIMALS);
-    String robot = null;
-    // Decide if we want to add a robot to the team.
-    if (random.nextInt(ROBOT_PROBABILITY) == 0) {
-      robot = "Robot-" + random.nextInt(NUM_ROBOTS);
-    }
-    // Create the new team.
-    TeamInfo newTeam = new TeamInfo(teamName, System.currentTimeMillis(), robot);
-    liveTeams.add(newTeam);
-    System.out.println("[+" + newTeam + "]");
-    return newTeam;
-  }
-
-  /**
-   * Remove a specific team.
-   */
-  private static synchronized void removeTeam(int teamIndex) {
-    TeamInfo removedTeam = liveTeams.remove(teamIndex);
-    System.out.println("[-" + removedTeam + "]");
-  }
-
-  /** Generate a user gaming event. */
-  private static String generateEvent(Long currTime, int delayInMillis) {
-    TeamInfo team = randomTeam(liveTeams);
-    String teamName = team.getTeamName();
-    String user;
-    final int parseErrorRate = 900000;
-
-    String robot = team.getRobot();
-    // If the team has an associated robot team member...
-    if (robot != null) {
-      // Then use that robot for the message with some probability.
-      // Set this probability to higher than that used to select any of the 'regular' team
-      // members, so that if there is a robot on the team, it has a higher click rate.
-      if (random.nextInt(team.numMembers() / 2) == 0) {
-        user = robot;
-      } else {
-        user = team.getRandomUser();
-      }
-    } else { // No robot.
-      user = team.getRandomUser();
-    }
-    String event = user + "," + teamName + "," + random.nextInt(MAX_SCORE);
-    // Randomly introduce occasional parse errors. You can see a custom counter tracking the number
-    // of such errors in the Dataflow Monitoring UI, as the example pipeline runs.
-    if (random.nextInt(parseErrorRate) == 0) {
-      System.out.println("Introducing a parse error.");
-      event = "THIS LINE REPRESENTS CORRUPT DATA AND WILL CAUSE A PARSE ERROR";
-    }
-    return addTimeInfoToEvent(event, currTime, delayInMillis);
-  }
-
-  /**
-   * Add time info to a generated gaming event.
-   */
-  private static String addTimeInfoToEvent(String message, Long currTime, int delayInMillis) {
-    String eventTimeString =
-        Long.toString((currTime - delayInMillis) / 1000 * 1000);
-    // Add a (redundant) 'human-readable' date string to make the data semantics more clear.
-    String dateString = fmt.print(currTime);
-    message = message + "," + eventTimeString + "," + dateString;
-    return message;
-  }
-
-  /**
-   * Publish 'numMessages' arbitrary events from live users with the provided delay, to a
-   * PubSub topic.
-   */
-  public static void publishData(int numMessages, int delayInMillis)
-      throws IOException {
-    List<PubsubMessage> pubsubMessages = new ArrayList<>();
-
-    for (int i = 0; i < Math.max(1, numMessages); i++) {
-      Long currTime = System.currentTimeMillis();
-      String message = generateEvent(currTime, delayInMillis);
-      PubsubMessage pubsubMessage = new PubsubMessage()
-              .encodeData(message.getBytes("UTF-8"));
-      pubsubMessage.setAttributes(
-          ImmutableMap.of(TIMESTAMP_ATTRIBUTE,
-              Long.toString((currTime - delayInMillis) / 1000 * 1000)));
-      if (delayInMillis != 0) {
-        System.out.println(pubsubMessage.getAttributes());
-        System.out.println("late data for: " + message);
-      }
-      pubsubMessages.add(pubsubMessage);
-    }
-
-    PublishRequest publishRequest = new PublishRequest();
-    publishRequest.setMessages(pubsubMessages);
-    pubsub.projects().topics().publish(topic, publishRequest).execute();
-  }
-
-  /**
-   * Publish generated events to a file.
-   */
-  public static void publishDataToFile(String fileName, int numMessages, int delayInMillis)
-      throws IOException {
-    PrintWriter out = new PrintWriter(new OutputStreamWriter(
-        new BufferedOutputStream(new FileOutputStream(fileName, true)), "UTF-8"));
-
-    try {
-      for (int i = 0; i < Math.max(1, numMessages); i++) {
-        Long currTime = System.currentTimeMillis();
-        String message = generateEvent(currTime, delayInMillis);
-        out.println(message);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    } finally {
-      if (out != null) {
-        out.flush();
-        out.close();
-      }
-    }
-  }
-
-
-  public static void main(String[] args) throws IOException, InterruptedException {
-    if (args.length < 3) {
-      System.out.println("Usage: Injector project-name (topic-name|none) (filename|none)");
-      System.exit(1);
-    }
-    boolean writeToFile = false;
-    boolean writeToPubsub = true;
-    project = args[0];
-    String topicName = args[1];
-    String fileName = args[2];
-    // The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if
-    // specified; otherwise, it will try to write to a file.
-    if (topicName.equalsIgnoreCase("none")) {
-      writeToFile = true;
-      writeToPubsub = false;
-    }
-    if (writeToPubsub) {
-      // Create the PubSub client.
-      pubsub = InjectorUtils.getClient();
-      // Create the PubSub topic as necessary.
-      topic = InjectorUtils.getFullyQualifiedTopicName(project, topicName);
-      InjectorUtils.createTopic(pubsub, topic);
-      System.out.println("Injecting to topic: " + topic);
-    } else {
-      if (fileName.equalsIgnoreCase("none")) {
-        System.out.println("Filename not specified.");
-        System.exit(1);
-      }
-      System.out.println("Writing to file: " + fileName);
-    }
-    System.out.println("Starting Injector");
-
-    // Start off with some random live teams.
-    while (liveTeams.size() < NUM_LIVE_TEAMS) {
-      addLiveTeam();
-    }
-
-    // Publish messages at a rate determined by the QPS and Thread sleep settings.
-    for (int i = 0; true; i++) {
-      if (Thread.activeCount() > 10) {
-        System.err.println("I'm falling behind!");
-      }
-
-      // Decide if this should be a batch of late data.
-      final int numMessages;
-      final int delayInMillis;
-      if (i % LATE_DATA_RATE == 0) {
-        // Insert delayed data for one user (one message only)
-        delayInMillis = BASE_DELAY_IN_MILLIS + random.nextInt(FUZZY_DELAY_IN_MILLIS);
-        numMessages = 1;
-        System.out.println("DELAY(" + delayInMillis + ", " + numMessages + ")");
-      } else {
-        System.out.print(".");
-        delayInMillis = 0;
-        numMessages = MIN_QPS + random.nextInt(QPS_RANGE);
-      }
-
-      if (writeToFile) { // Won't use threading for the file write.
-        publishDataToFile(fileName, numMessages, delayInMillis);
-      } else { // Write to PubSub.
-        // Start a thread to inject some data.
-        new Thread(){
-          @Override
-          public void run() {
-            try {
-              publishData(numMessages, delayInMillis);
-            } catch (IOException e) {
-              System.err.println(e);
-            }
-          }
-        }.start();
-      }
-
-      // Wait before creating another injector thread.
-      Thread.sleep(THREAD_SLEEP_MS);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java
deleted file mode 100644
index db58650..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java
+++ /dev/null
@@ -1,102 +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 com.google.cloud.dataflow.examples.complete.game.injector;
-
-
-import com.google.api.client.googleapis.auth.oauth2.GoogleCredential;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.googleapis.util.Utils;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.client.http.HttpStatusCodes;
-import com.google.api.client.http.HttpTransport;
-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;
-
-class InjectorUtils {
-
-  private static final String APP_NAME = "injector";
-
-  /**
-   * Builds a new Pubsub client and returns it.
-   */
-  public static Pubsub getClient(final HttpTransport httpTransport,
-                                 final JsonFactory jsonFactory)
-           throws IOException {
-      Preconditions.checkNotNull(httpTransport);
-      Preconditions.checkNotNull(jsonFactory);
-      GoogleCredential credential =
-          GoogleCredential.getApplicationDefault(httpTransport, jsonFactory);
-      if (credential.createScopedRequired()) {
-          credential = credential.createScoped(PubsubScopes.all());
-      }
-      if (credential.getClientAuthentication() != null) {
-        System.out.println("\n***Warning! You are not using service account credentials to "
-          + "authenticate.\nYou need to use service account credentials for this example,"
-          + "\nsince user-level credentials do not have enough pubsub quota,\nand so you will run "
-          + "out of PubSub quota very quickly.\nSee "
-          + "https://developers.google.com/identity/protocols/application-default-credentials.");
-        System.exit(1);
-      }
-      HttpRequestInitializer initializer =
-          new RetryHttpInitializerWrapper(credential);
-      return new Pubsub.Builder(httpTransport, jsonFactory, initializer)
-              .setApplicationName(APP_NAME)
-              .build();
-  }
-
-  /**
-   * Builds a new Pubsub client with default HttpTransport and
-   * JsonFactory and returns it.
-   */
-  public static Pubsub getClient() throws IOException {
-      return getClient(Utils.getDefaultTransport(),
-                       Utils.getDefaultJsonFactory());
-  }
-
-
-  /**
-   * Returns the fully qualified topic name for Pub/Sub.
-   */
-  public static String getFullyQualifiedTopicName(
-          final String project, final String topic) {
-      return String.format("projects/%s/topics/%s", project, topic);
-  }
-
-  /**
-   * Create a topic if it doesn't exist.
-   */
-  public static void createTopic(Pubsub client, String fullTopicName)
-      throws IOException {
-    try {
-        client.projects().topics().get(fullTopicName).execute();
-    } catch (GoogleJsonResponseException e) {
-      if (e.getStatusCode() == HttpStatusCodes.STATUS_CODE_NOT_FOUND) {
-        Topic topic = client.projects().topics()
-                .create(fullTopicName, new Topic())
-                .execute();
-        System.out.printf("Topic %s was created.\n", topic.getName());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java
deleted file mode 100644
index 9d58837..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java
+++ /dev/null
@@ -1,129 +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 com.google.cloud.dataflow.examples.complete.game.injector;
-
-import com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.http.HttpBackOffIOExceptionHandler;
-import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler;
-import com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpRequestInitializer;
-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;
-
-/**
- * RetryHttpInitializerWrapper will automatically retry upon RPC
- * failures, preserving the auto-refresh behavior of the Google
- * Credentials.
- */
-public class RetryHttpInitializerWrapper implements HttpRequestInitializer {
-
-    /**
-     * A private logger.
-     */
-    private static final Logger LOG =
-            Logger.getLogger(RetryHttpInitializerWrapper.class.getName());
-
-    /**
-     * One minutes in miliseconds.
-     */
-    private static final int ONEMINITUES = 60000;
-
-    /**
-     * Intercepts the request for filling in the "Authorization"
-     * header field, as well as recovering from certain unsuccessful
-     * error codes wherein the Credential must refresh its token for a
-     * retry.
-     */
-    private final Credential wrappedCredential;
-
-    /**
-     * A sleeper; you can replace it with a mock in your test.
-     */
-    private final Sleeper sleeper;
-
-    /**
-     * A constructor.
-     *
-     * @param wrappedCredential Credential which will be wrapped and
-     * used for providing auth header.
-     */
-    public RetryHttpInitializerWrapper(final Credential wrappedCredential) {
-        this(wrappedCredential, Sleeper.DEFAULT);
-    }
-
-    /**
-     * A protected constructor only for testing.
-     *
-     * @param wrappedCredential Credential which will be wrapped and
-     * used for providing auth header.
-     * @param sleeper Sleeper for easy testing.
-     */
-    RetryHttpInitializerWrapper(
-            final Credential wrappedCredential, final Sleeper sleeper) {
-        this.wrappedCredential = Preconditions.checkNotNull(wrappedCredential);
-        this.sleeper = sleeper;
-    }
-
-    /**
-     * Initializes the given request.
-     */
-    @Override
-    public final void initialize(final HttpRequest request) {
-        request.setReadTimeout(2 * ONEMINITUES); // 2 minutes read timeout
-        final HttpUnsuccessfulResponseHandler backoffHandler =
-                new HttpBackOffUnsuccessfulResponseHandler(
-                        new ExponentialBackOff())
-                        .setSleeper(sleeper);
-        request.setInterceptor(wrappedCredential);
-        request.setUnsuccessfulResponseHandler(
-                new HttpUnsuccessfulResponseHandler() {
-                    @Override
-                    public boolean handleResponse(
-                            final HttpRequest request,
-                            final HttpResponse response,
-                            final boolean supportsRetry) throws IOException {
-                        if (wrappedCredential.handleResponse(
-                                request, response, supportsRetry)) {
-                            // If credential decides it can handle it,
-                            // the return code or message indicated
-                            // something specific to authentication,
-                            // and no backoff is desired.
-                            return true;
-                        } else if (backoffHandler.handleResponse(
-                                request, response, supportsRetry)) {
-                            // Otherwise, we defer to the judgement of
-                            // our internal backoff handler.
-                            LOG.info("Retrying "
-                                    + request.getUrl().toString());
-                            return true;
-                        } else {
-                            return false;
-                        }
-                    }
-                });
-        request.setIOExceptionHandler(
-                new HttpBackOffIOExceptionHandler(new ExponentialBackOff())
-                        .setSleeper(sleeper));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteToBigQuery.java
deleted file mode 100644
index 4bcfb72..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteToBigQuery.java
+++ /dev/null
@@ -1,135 +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 com.google.cloud.dataflow.examples.complete.game.utils;
-
-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.cloud.dataflow.examples.complete.game.UserScore;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Generate, format, and write BigQuery table row information. Use provided information about
- * the field names and types, as well as lambda functions that describe how to generate their
- * values.
- */
-public class WriteToBigQuery<T>
-    extends PTransform<PCollection<T>, PDone> {
-
-  protected String tableName;
-  protected Map<String, FieldInfo<T>> fieldInfo;
-
-  public WriteToBigQuery() {
-  }
-
-  public WriteToBigQuery(String tableName,
-      Map<String, FieldInfo<T>> fieldInfo) {
-    this.tableName = tableName;
-    this.fieldInfo = fieldInfo;
-  }
-
-  /** Define a class to hold information about output table field definitions. */
-  public static class FieldInfo<T> implements Serializable {
-    // The BigQuery 'type' of the field
-    private String fieldType;
-    // A lambda function to generate the field value
-    private SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fieldFn;
-
-    public FieldInfo(String fieldType,
-        SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fieldFn) {
-      this.fieldType = fieldType;
-      this.fieldFn = fieldFn;
-    }
-
-    String getFieldType() {
-      return this.fieldType;
-    }
-
-    SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> getFieldFn() {
-      return this.fieldFn;
-    }
-  }
-  /** Convert each key/score pair into a BigQuery TableRow as specified by fieldFn. */
-  protected class BuildRowFn extends DoFn<T, TableRow> {
-
-    @Override
-    public void processElement(ProcessContext c) {
-
-      TableRow row = new TableRow();
-      for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
-          String key = entry.getKey();
-          FieldInfo<T> fcnInfo = entry.getValue();
-          SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fcn =
-            fcnInfo.getFieldFn();
-          row.set(key, fcn.apply(c));
-        }
-      c.output(row);
-    }
-  }
-
-  /** Build the output table schema. */
-  protected TableSchema getSchema() {
-    List<TableFieldSchema> fields = new ArrayList<>();
-    for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
-      String key = entry.getKey();
-      FieldInfo<T> fcnInfo = entry.getValue();
-      String bqType = fcnInfo.getFieldType();
-      fields.add(new TableFieldSchema().setName(key).setType(bqType));
-    }
-    return new TableSchema().setFields(fields);
-  }
-
-  @Override
-  public PDone apply(PCollection<T> teamAndScore) {
-    return teamAndScore
-      .apply(ParDo.named("ConvertToRow").of(new BuildRowFn()))
-      .apply(BigQueryIO.Write
-                .to(getTable(teamAndScore.getPipeline(),
-                    tableName))
-                .withSchema(getSchema())
-                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
-                .withWriteDisposition(WriteDisposition.WRITE_APPEND));
-  }
-
-  /** Utility to construct an output table reference. */
-  static TableReference getTable(Pipeline pipeline, String tableName) {
-    PipelineOptions options = pipeline.getOptions();
-    TableReference table = new TableReference();
-    table.setDatasetId(options.as(UserScore.Options.class).getDataset());
-    table.setProjectId(options.as(GcpOptions.class).getProject());
-    table.setTableId(tableName);
-    return table;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteWindowedToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteWindowedToBigQuery.java
deleted file mode 100644
index 41257ca..0000000
--- a/examples/java8/src/main/java/com/google/cloud/dataflow/examples/complete/game/utils/WriteWindowedToBigQuery.java
+++ /dev/null
@@ -1,77 +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 com.google.cloud.dataflow.examples.complete.game.utils;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-import java.util.Map;
-
-/**
- * Generate, format, and write BigQuery table row information. Subclasses {@link WriteToBigQuery}
- * to require windowing; so this subclass may be used for writes that require access to the
- * context's window information.
- */
-public class WriteWindowedToBigQuery<T>
-    extends WriteToBigQuery<T> {
-
-  public WriteWindowedToBigQuery(String tableName,
-      Map<String, FieldInfo<T>> fieldInfo) {
-    super(tableName, fieldInfo);
-  }
-
-  /** Convert each key/score pair into a BigQuery TableRow. */
-  protected class BuildRowFn extends DoFn<T, TableRow>
-      implements RequiresWindowAccess {
-
-    @Override
-    public void processElement(ProcessContext c) {
-
-      TableRow row = new TableRow();
-      for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
-          String key = entry.getKey();
-          FieldInfo<T> fcnInfo = entry.getValue();
-          SerializableFunction<DoFn<T, TableRow>.ProcessContext, Object> fcn =
-            fcnInfo.getFieldFn();
-          row.set(key, fcn.apply(c));
-        }
-      c.output(row);
-    }
-  }
-
-  @Override
-  public PDone apply(PCollection<T> teamAndScore) {
-    return teamAndScore
-      .apply(ParDo.named("ConvertToRow").of(new BuildRowFn()))
-      .apply(BigQueryIO.Write
-                .to(getTable(teamAndScore.getPipeline(),
-                    tableName))
-                .withSchema(getSchema())
-                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
-                .withWriteDisposition(WriteDisposition.WRITE_APPEND));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..8705ed3
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -0,0 +1,69 @@
+/*
+ * 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 com.google.cloud.dataflow.examples;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.FlatMapElements;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import java.util.Arrays;
+
+/**
+ * An example that counts words in Shakespeare, using Java 8 language features.
+ *
+ * <p>See {@link MinimalWordCount} for a comprehensive explanation.
+ */
+public class MinimalWordCountJava8 {
+
+  public static void main(String[] args) {
+    DataflowPipelineOptions options = PipelineOptionsFactory.create()
+        .as(DataflowPipelineOptions.class);
+
+    options.setRunner(BlockingDataflowPipelineRunner.class);
+
+    // CHANGE 1 of 3: Your project ID is required in order to run your pipeline on the Google Cloud.
+    options.setProject("SET_YOUR_PROJECT_ID_HERE");
+
+    // CHANGE 2 of 3: Your Google Cloud Storage path is required for staging local files.
+    options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY");
+
+    Pipeline p = Pipeline.create(options);
+
+    p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
+     .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
+         .withOutputType(new TypeDescriptor<String>() {}))
+     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
+     .apply(Count.<String>perElement())
+     .apply(MapElements
+         .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())
+         .withOutputType(new TypeDescriptor<String>() {}))
+
+     // CHANGE 3 of 3: The Google Cloud Storage path is required for outputting the results to.
+     .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..89832b2
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
@@ -0,0 +1,340 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.Mean;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.Values;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+
+/**
+ * This class is the fourth in a series of four pipelines that tell a story in a 'gaming'
+ * domain, following {@link UserScore}, {@link HourlyTeamScore}, and {@link LeaderBoard}.
+ * New concepts: session windows and finding session duration; use of both
+ * singleton and non-singleton side inputs.
+ *
+ * <p> This pipeline builds on the {@link LeaderBoard} functionality, and adds some "business
+ * intelligence" analysis: abuse detection and usage patterns. The pipeline derives the Mean user
+ * score sum for a window, and uses that information to identify likely spammers/robots. (The robots
+ * have a higher click rate than the human users). The 'robot' users are then filtered out when
+ * calculating the team scores.
+ *
+ * <p> Additionally, user sessions are tracked: that is, we find bursts of user activity using
+ * session windows. Then, the mean session duration information is recorded in the context of
+ * subsequent fixed windowing. (This could be used to tell us what games are giving us greater
+ * user retention).
+ *
+ * <p> Run {@code com.google.cloud.dataflow.examples.complete.game.injector.Injector} to generate
+ * pubsub data for this pipeline. The {@code Injector} documentation provides more detail.
+ *
+ * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
+ * like this:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ *   --dataset=YOUR-DATASET
+ *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
+ * }
+ * </pre>
+ * where the BigQuery dataset you specify must already exist. The PubSub topic you specify should
+ * be the same topic to which the Injector is publishing.
+ */
+public class GameStats extends LeaderBoard {
+
+  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
+
+  private static DateTimeFormatter fmt =
+      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
+          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
+
+  /**
+   * Filter out all but those users with a high clickrate, which we will consider as 'spammy' uesrs.
+   * We do this by finding the mean total score per user, then using that information as a side
+   * input to filter out all but those user scores that are > (mean * SCORE_WEIGHT)
+   */
+  // [START DocInclude_AbuseDetect]
+  public static class CalculateSpammyUsers
+      extends PTransform<PCollection<KV<String, Integer>>, PCollection<KV<String, Integer>>> {
+    private static final Logger LOG = LoggerFactory.getLogger(CalculateSpammyUsers.class);
+    private static final double SCORE_WEIGHT = 2.5;
+
+    @Override
+    public PCollection<KV<String, Integer>> apply(PCollection<KV<String, Integer>> userScores) {
+
+      // Get the sum of scores for each user.
+      PCollection<KV<String, Integer>> sumScores = userScores
+          .apply("UserSum", Sum.<String>integersPerKey());
+
+      // Extract the score from each element, and use it to find the global mean.
+      final PCollectionView<Double> globalMeanScore = sumScores.apply(Values.<Integer>create())
+          .apply(Mean.<Integer>globally().asSingletonView());
+
+      // Filter the user sums using the global mean.
+      PCollection<KV<String, Integer>> filtered = sumScores
+          .apply(ParDo
+              .named("ProcessAndFilter")
+              // use the derived mean total score as a side input
+              .withSideInputs(globalMeanScore)
+              .of(new DoFn<KV<String, Integer>, KV<String, Integer>>() {
+                private final Aggregator<Long, Long> numSpammerUsers =
+                  createAggregator("SpammerUsers", new Sum.SumLongFn());
+                @Override
+                public void processElement(ProcessContext c) {
+                  Integer score = c.element().getValue();
+                  Double gmc = c.sideInput(globalMeanScore);
+                  if (score > (gmc * SCORE_WEIGHT)) {
+                    LOG.info("user " + c.element().getKey() + " spammer score " + score
+                        + " with mean " + gmc);
+                    numSpammerUsers.addValue(1L);
+                    c.output(c.element());
+                  }
+                }
+              }));
+      return filtered;
+    }
+  }
+  // [END DocInclude_AbuseDetect]
+
+  /**
+   * Calculate and output an element's session duration.
+   */
+  private static class UserSessionInfoFn extends DoFn<KV<String, Integer>, Integer>
+      implements RequiresWindowAccess {
+
+    @Override
+    public void processElement(ProcessContext c) {
+      IntervalWindow w = (IntervalWindow) c.window();
+      int duration = new Duration(
+          w.start(), w.end()).toPeriod().toStandardMinutes().getMinutes();
+      c.output(duration);
+    }
+  }
+
+
+  /**
+   * Options supported by {@link GameStats}.
+   */
+  static interface Options extends LeaderBoard.Options {
+    @Description("Numeric value of fixed window duration for user analysis, in minutes")
+    @Default.Integer(60)
+    Integer getFixedWindowDuration();
+    void setFixedWindowDuration(Integer value);
+
+    @Description("Numeric value of gap between user sessions, in minutes")
+    @Default.Integer(5)
+    Integer getSessionGap();
+    void setSessionGap(Integer value);
+
+    @Description("Numeric value of fixed window for finding mean of user session duration, "
+        + "in minutes")
+    @Default.Integer(30)
+    Integer getUserActivityWindowDuration();
+    void setUserActivityWindowDuration(Integer value);
+
+    @Description("Prefix used for the BigQuery table names")
+    @Default.String("game_stats")
+    String getTablePrefix();
+    void setTablePrefix(String value);
+  }
+
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is used to write information about team score sums.
+   */
+  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
+      configureWindowedWrite() {
+    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
+        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
+    tableConfigure.put("team",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+            c -> c.element().getKey()));
+    tableConfigure.put("total_score",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
+            c -> c.element().getValue()));
+    tableConfigure.put("window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+          c -> { IntervalWindow w = (IntervalWindow) c.window();
+                 return fmt.print(w.start()); }));
+    tableConfigure.put("processing_time",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", c -> fmt.print(Instant.now())));
+    return tableConfigure;
+  }
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is used to write information about mean user session time.
+   */
+  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<Double>>
+      configureSessionWindowWrite() {
+
+    Map<String, WriteWindowedToBigQuery.FieldInfo<Double>> tableConfigure =
+        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<Double>>();
+    tableConfigure.put("window_start",
+        new WriteWindowedToBigQuery.FieldInfo<Double>("STRING",
+          c -> { IntervalWindow w = (IntervalWindow) c.window();
+                 return fmt.print(w.start()); }));
+    tableConfigure.put("mean_duration",
+        new WriteWindowedToBigQuery.FieldInfo<Double>("FLOAT", c -> c.element()));
+    return tableConfigure;
+  }
+
+
+
+  public static void main(String[] args) throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    // Enforce that this pipeline is always run in streaming mode.
+    options.setStreaming(true);
+    // Allow the pipeline to be cancelled automatically.
+    options.setRunner(DataflowPipelineRunner.class);
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
+    Pipeline pipeline = Pipeline.create(options);
+
+    // Read Events from Pub/Sub using custom timestamps
+    PCollection<GameActionInfo> rawEvents = pipeline
+        .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic()))
+        .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()));
+
+    // Extract username/score pairs from the event stream
+    PCollection<KV<String, Integer>> userEvents =
+        rawEvents.apply("ExtractUserScore",
+          MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore()))
+            .withOutputType(new TypeDescriptor<KV<String, Integer>>() {}));
+
+    // 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())))
+          )
+
+      // Filter out everyone but those with (SCORE_WEIGHT * avg) clickrate.
+      // These might be robots/spammers.
+      .apply("CalculateSpammyUsers", new CalculateSpammyUsers())
+      // Derive a view from the collection of spammer users. It will be used as a side input
+      // in calculating the team score sums, below.
+      .apply("CreateSpammersView", View.<String, Integer>asMap());
+
+    // [START DocInclude_FilterAndCalc]
+    // Calculate the total score per team over fixed windows,
+    // and emit cumulative updates for late data. Uses the side input derived above-- the set of
+    // 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())))
+          )
+      // Filter out the detected spammer users, using the side input derived above.
+      .apply(ParDo.named("FilterOutSpammers")
+              .withSideInputs(spammersView)
+              .of(new DoFn<GameActionInfo, GameActionInfo>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  // If the user is not in the spammers Map, output the data element.
+                  if (c.sideInput(spammersView).get(c.element().getUser().trim()) == null) {
+                    c.output(c.element());
+                  }
+                }
+              }))
+      // Extract and sum teamname/score pairs from the event data.
+      .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
+      // [END DocInclude_FilterAndCalc]
+      // Write the result to BigQuery
+      .apply("WriteTeamSums",
+             new WriteWindowedToBigQuery<KV<String, Integer>>(
+                options.getTablePrefix() + "_team", configureWindowedWrite()));
+
+
+    // [START DocInclude_SessionCalc]
+    // Detect user sessions-- that is, a burst of activity separated by a gap from further
+    // activity. Find and record the mean session lengths.
+    // 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()))
+      // 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))
+      // Get the duration per session.
+      .apply("UserSessionActivity", ParDo.of(new UserSessionInfoFn()))
+      // [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()))))
+      // Find the mean session duration in each window.
+      .apply(Mean.<Integer>globally().withoutDefaults())
+      // Write this info to a BigQuery table.
+      .apply("WriteAvgSessionLength",
+             new WriteWindowedToBigQuery<Double>(
+                options.getTablePrefix() + "_sessions", configureSessionWindowWrite()));
+    // [END DocInclude_Rewindow]
+
+
+    // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
+    // command line.
+    PipelineResult result = pipeline.run();
+    dataflowUtils.waitToFinish(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..58944c5
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -0,0 +1,194 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.Filter;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.WithTimestamps;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+
+/**
+ * This class is the second in a series of four pipelines that tell a story in a 'gaming'
+ * domain, following {@link UserScore}. In addition to the concepts introduced in {@link UserScore},
+ * new concepts include: windowing and element timestamps; use of {@code Filter.byPredicate()}.
+ *
+ * <p> This pipeline processes data collected from gaming events in batch, building on {@link
+ * UserScore} but using fixed windows. It calculates the sum of scores per team, for each window,
+ * optionally allowing specification of two timestamps before and after which data is filtered out.
+ * This allows a model where late data collected after the intended analysis window can be included,
+ * and any late-arriving data prior to the beginning of the analysis window can be removed as well.
+ * By using windowing and adding element timestamps, we can do finer-grained analysis than with the
+ * {@link UserScore} pipeline. However, our batch processing is high-latency, in that we don't get
+ * results from plays at the beginning of the batch's time period until the batch is processed.
+ *
+ * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
+ * like this:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ *   --dataset=YOUR-DATASET
+ * }
+ * </pre>
+ * where the BigQuery dataset you specify must already exist.
+ *
+ * <p> Optionally include {@code --input} to specify the batch input file path.
+ * To indicate a time after which the data should be filtered out, include the
+ * {@code --stopMin} arg. E.g., {@code --stopMin=2015-10-18-23-59} indicates that any data
+ * timestamped after 23:59 PST on 2015-10-18 should not be included in the analysis.
+ * To indicate a time before which data should be filtered out, include the {@code --startMin} arg.
+ * If you're using the default input specified in {@link UserScore},
+ * "gs://dataflow-samples/game/gaming_data*.csv", then
+ * {@code --startMin=2015-11-16-16-10 --stopMin=2015-11-17-16-10} are good values.
+ */
+public class HourlyTeamScore extends UserScore {
+
+  private static DateTimeFormatter fmt =
+      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
+          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
+  private static DateTimeFormatter minFmt =
+      DateTimeFormat.forPattern("yyyy-MM-dd-HH-mm")
+          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
+
+
+  /**
+   * Options supported by {@link HourlyTeamScore}.
+   */
+  static interface Options extends UserScore.Options {
+
+    @Description("Numeric value of fixed window duration, in minutes")
+    @Default.Integer(60)
+    Integer getWindowDuration();
+    void setWindowDuration(Integer value);
+
+    @Description("String representation of the first minute after which to generate results,"
+        + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST."
+        + "Any input data timestamped prior to that minute won't be included in the sums.")
+    @Default.String("1970-01-01-00-00")
+    String getStartMin();
+    void setStartMin(String value);
+
+    @Description("String representation of the first minute for which to not generate results,"
+        + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST."
+        + "Any input data timestamped after that minute won't be included in the sums.")
+    @Default.String("2100-01-01-00-00")
+    String getStopMin();
+    void setStopMin(String value);
+
+    @Description("The BigQuery table name. Should not already exist.")
+    @Default.String("hourly_team_score")
+    String getTableName();
+    void setTableName(String value);
+  }
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is passed to the {@link WriteWindowedToBigQuery} constructor to write team score sums and
+   * includes information about window start time.
+   */
+  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
+      configureWindowedTableWrite() {
+    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfig =
+        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
+    tableConfig.put("team",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+            c -> c.element().getKey()));
+    tableConfig.put("total_score",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
+            c -> c.element().getValue()));
+    tableConfig.put("window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+          c -> { IntervalWindow w = (IntervalWindow) c.window();
+                 return fmt.print(w.start()); }));
+    return tableConfig;
+  }
+
+
+  /**
+   * Run a batch pipeline to do windowed analysis of the data.
+   */
+  // [START DocInclude_HTSMain]
+  public static void main(String[] args) throws Exception {
+    // Begin constructing a pipeline configured by commandline flags.
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline pipeline = Pipeline.create(options);
+
+    final Instant stopMinTimestamp = new Instant(minFmt.parseMillis(options.getStopMin()));
+    final Instant startMinTimestamp = new Instant(minFmt.parseMillis(options.getStartMin()));
+
+    // Read 'gaming' events from a text file.
+    pipeline.apply(TextIO.Read.from(options.getInput()))
+      // Parse the incoming data.
+      .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
+
+      // Filter out data before and after the given times so that it is not included
+      // in the calculations. As we collect data in batches (say, by day), the batch for the day
+      // that we want to analyze could potentially include some late-arriving data from the previous
+      // day. If so, we want to weed it out. Similarly, if we include data from the following day
+      // (to scoop up late-arriving events from the day we're analyzing), we need to weed out events
+      // that fall after the time period we want to analyze.
+      // [START DocInclude_HTSFilters]
+      .apply("FilterStartTime", Filter.byPredicate(
+          (GameActionInfo gInfo)
+              -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
+      .apply("FilterEndTime", Filter.byPredicate(
+          (GameActionInfo gInfo)
+              -> gInfo.getTimestamp() < stopMinTimestamp.getMillis()))
+      // [END DocInclude_HTSFilters]
+
+      // [START DocInclude_HTSAddTsAndWindow]
+      // 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()))))
+      // [END DocInclude_HTSAddTsAndWindow]
+
+      // Extract and sum teamname/score pairs from the event data.
+      .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
+      .apply("WriteTeamScoreSums",
+        new WriteWindowedToBigQuery<KV<String, Integer>>(options.getTableName(),
+            configureWindowedTableWrite()));
+
+
+    pipeline.run();
+  }
+  // [END DocInclude_HTSMain]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..cedd696
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -0,0 +1,238 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.complete.game;
+
+import com.google.cloud.dataflow.examples.common.DataflowExampleOptions;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.complete.game.utils.WriteToBigQuery;
+import com.google.cloud.dataflow.examples.complete.game.utils.WriteWindowedToBigQuery;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+
+/**
+ * This class is the third in a series of four pipelines that tell a story in a 'gaming' domain,
+ * following {@link UserScore} and {@link HourlyTeamScore}. Concepts include: processing unbounded
+ * data using fixed windows; use of custom timestamps and event-time processing; generation of
+ * early/speculative results; using .accumulatingFiredPanes() to do cumulative processing of late-
+ * arriving data.
+ *
+ * <p> This pipeline processes an unbounded stream of 'game events'. The calculation of the team
+ * scores uses fixed windowing based on event time (the time of the game play event), not
+ * processing time (the time that an event is processed by the pipeline). The pipeline calculates
+ * the sum of scores per team, for each window. By default, the team scores are calculated using
+ * one-hour windows.
+ *
+ * <p> In contrast-- to demo another windowing option-- the user scores are calculated using a
+ * global window, which periodically (every ten minutes) emits cumulative user score sums.
+ *
+ * <p> In contrast to the previous pipelines in the series, which used static, finite input data,
+ * here we're using an unbounded data source, which lets us provide speculative results, and allows
+ * handling of late data, at much lower latency. We can use the early/speculative results to keep a
+ * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct
+ * results, e.g. for 'team prizes'. We're now outputing window results as they're
+ * calculated, giving us much lower latency than with the previous batch examples.
+ *
+ * <p> Run {@link injector.Injector} to generate pubsub data for this pipeline.  The Injector
+ * documentation provides more detail on how to do this.
+ *
+ * <p> To execute this pipeline using the Dataflow service, specify the pipeline configuration
+ * like this:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ *   --dataset=YOUR-DATASET
+ *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
+ * }
+ * </pre>
+ * where the BigQuery dataset you specify must already exist.
+ * The PubSub topic you specify should be the same topic to which the Injector is publishing.
+ */
+public class LeaderBoard extends HourlyTeamScore {
+
+  private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms";
+
+  private static DateTimeFormatter fmt =
+      DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")
+          .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")));
+  static final Duration FIVE_MINUTES = Duration.standardMinutes(5);
+  static final Duration TEN_MINUTES = Duration.standardMinutes(10);
+
+
+  /**
+   * Options supported by {@link LeaderBoard}.
+   */
+  static interface Options extends HourlyTeamScore.Options, DataflowExampleOptions {
+
+    @Description("Pub/Sub topic to read from")
+    @Validation.Required
+    String getTopic();
+    void setTopic(String value);
+
+    @Description("Numeric value of fixed window duration for team analysis, in minutes")
+    @Default.Integer(60)
+    Integer getTeamWindowDuration();
+    void setTeamWindowDuration(Integer value);
+
+    @Description("Numeric value of allowed data lateness, in minutes")
+    @Default.Integer(120)
+    Integer getAllowedLateness();
+    void setAllowedLateness(Integer value);
+
+    @Description("Prefix used for the BigQuery table names")
+    @Default.String("leaderboard")
+    String getTableName();
+    void setTableName(String value);
+  }
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is used to write team score sums and includes event timing information.
+   */
+  protected static Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>
+      configureWindowedTableWrite() {
+
+    Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
+        new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
+    tableConfigure.put("team",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+            c -> c.element().getKey()));
+    tableConfigure.put("total_score",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
+            c -> c.element().getValue()));
+    tableConfigure.put("window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
+          c -> { IntervalWindow w = (IntervalWindow) c.window();
+                 return fmt.print(w.start()); }));
+    tableConfigure.put("processing_time",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", c -> fmt.print(Instant.now())));
+    tableConfigure.put("timing",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", c -> c.pane().getTiming().toString()));
+    return tableConfigure;
+  }
+
+  /**
+   * Create a map of information that describes how to write pipeline output to BigQuery. This map
+   * is used to write user score sums.
+   */
+  protected static Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>
+      configureGlobalWindowBigQueryWrite() {
+
+    Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
+        configureBigQueryWrite();
+    tableConfigure.put("processing_time",
+        new WriteToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", c -> fmt.print(Instant.now())));
+    return tableConfigure;
+  }
+
+
+  public static void main(String[] args) throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    // Enforce that this pipeline is always run in streaming mode.
+    options.setStreaming(true);
+    // For example purposes, allow the pipeline to be easily cancelled instead of running
+    // continuously.
+    options.setRunner(DataflowPipelineRunner.class);
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
+    Pipeline pipeline = Pipeline.create(options);
+
+    // Read game events from Pub/Sub using custom timestamps, which are extracted from the pubsub
+    // data elements, and parse the data.
+    PCollection<GameActionInfo> gameEvents = pipeline
+        .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic()))
+        .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()));
+
+    // [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())))
+          // We will get early (speculative) results as well as cumulative
+          // processing of late data.
+          .triggering(
+            AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane()
+                  .plusDelayOf(FIVE_MINUTES))
+            .withLateFirings(AfterProcessingTime.pastFirstElementInPane()
+                  .plusDelayOf(TEN_MINUTES)))
+          .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))
+          .accumulatingFiredPanes())
+        // Extract and sum teamname/score pairs from the event data.
+        .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
+        // Write the results to BigQuery.
+        .apply("WriteTeamScoreSums",
+               new WriteWindowedToBigQuery<KV<String, Integer>>(
+                  options.getTableName() + "_team", configureWindowedTableWrite()));
+    // [END DocInclude_WindowAndTrigger]
+
+    // [START DocInclude_ProcTimeTrigger]
+    // 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())
+          // Get periodic results every ten minutes.
+              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
+                  .plusDelayOf(TEN_MINUTES)))
+              .accumulatingFiredPanes()
+              .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())))
+        // Extract and sum username/score pairs from the event data.
+        .apply("ExtractUserScore", new ExtractAndSumScore("user"))
+        // Write the results to BigQuery.
+        .apply("WriteUserScoreSums",
+               new WriteToBigQuery<KV<String, Integer>>(
+                  options.getTableName() + "_user", configureGlobalWindowBigQueryWrite()));
+    // [END DocInclude_ProcTimeTrigger]
+
+    // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
+    // command line.
+    PipelineResult result = pipeline.run();
+    dataflowUtils.waitToFinish(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java8/src/main/java/org/apache/beam/examples/complete/game/README.md
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/README.md b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/README.md
new file mode 100644
index 0000000..79b55ce
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/README.md
@@ -0,0 +1,113 @@
+
+# 'Gaming' examples
+
+
+This directory holds a series of example Dataflow pipelines in a simple 'mobile
+gaming' domain. They all require Java 8.  Each pipeline successively introduces
+new concepts, and gives some examples of using Java 8 syntax in constructing
+Dataflow pipelines. Other than usage of Java 8 lambda expressions, the concepts
+that are used apply equally well in Java 7.
+
+In the gaming scenario, many users play, as members of different teams, over
+the course of a day, and their actions are logged for processing. Some of the
+logged game events may be late-arriving, if users play on mobile devices and go
+transiently offline for a period.
+
+The scenario includes not only "regular" users, but "robot users", which have a
+higher click rate than the regular users, and may move from team to team.
+
+The first two pipelines in the series use pre-generated batch data samples. The
+second two pipelines read from a [PubSub](https://cloud.google.com/pubsub/)
+topic input.  For these examples, you will also need to run the
+`injector.Injector` program, which generates and publishes the gaming data to
+PubSub. The javadocs for each pipeline have more detailed information on how to
+run that pipeline.
+
+All of these pipelines write their results to BigQuery table(s).
+
+
+## The pipelines in the 'gaming' series
+
+### UserScore
+
+The first pipeline in the series is `UserScore`. This pipeline does batch
+processing of data collected from gaming events. It calculates the sum of
+scores per user, over an entire batch of gaming data (collected, say, for each
+day). The batch processing will not include any late data that arrives after
+the day's cutoff point.
+
+### HourlyTeamScore
+
+The next pipeline in the series is `HourlyTeamScore`. This pipeline also
+processes data collected from gaming events in batch. It builds on `UserScore`,
+but uses [fixed windows](https://cloud.google.com/dataflow/model/windowing), by
+default an hour in duration. It calculates the sum of scores per team, for each
+window, optionally allowing specification of two timestamps before and after
+which data is filtered out. This allows a model where late data collected after
+the intended analysis window can be included in the analysis, and any late-
+arriving data prior to the beginning of the analysis window can be removed as
+well.
+
+By using windowing and adding element timestamps, we can do finer-grained
+analysis than with the `UserScore` pipeline — we're now tracking scores for
+each hour rather than over the course of a whole day. However, our batch
+processing is high-latency, in that we don't get results from plays at the
+beginning of the batch's time period until the complete batch is processed.
+
+### LeaderBoard
+
+The third pipeline in the series is `LeaderBoard`. This pipeline processes an
+unbounded stream of 'game events' from a PubSub topic. The calculation of the
+team scores uses fixed windowing based on event time (the time of the game play
+event), not processing time (the time that an event is processed by the
+pipeline). The pipeline calculates the sum of scores per team, for each window.
+By default, the team scores are calculated using one-hour windows.
+
+In contrast — to demo another windowing option — the user scores are calculated
+using a global window, which periodically (every ten minutes) emits cumulative
+user score sums.
+
+In contrast to the previous pipelines in the series, which used static, finite
+input data, here we're using an unbounded data source, which lets us provide
+_speculative_ results, and allows handling of late data, at much lower latency.
+E.g., we could use the early/speculative results to keep a 'leaderboard'
+updated in near-realtime. Our handling of late data lets us generate correct
+results, e.g. for 'team prizes'. We're now outputing window results as they're
+calculated, giving us much lower latency than with the previous batch examples.
+
+### GameStats
+
+The fourth pipeline in the series is `GameStats`. This pipeline builds
+on the `LeaderBoard` functionality — supporting output of speculative and late
+data — and adds some "business intelligence" analysis: identifying abuse
+detection. The pipeline derives the Mean user score sum for a window, and uses
+that information to identify likely spammers/robots. (The injector is designed
+so that the "robots" have a higher click rate than the "real" users). The robot
+users are then filtered out when calculating the team scores.
+
+Additionally, user sessions are tracked: that is, we find bursts of user
+activity using session windows. Then, the mean session duration information is
+recorded in the context of subsequent fixed windowing. (This could be used to
+tell us what games are giving us greater user retention).
+
+### Running the PubSub Injector
+
+The `LeaderBoard` and `GameStats` example pipelines read unbounded data
+from a PubSub topic.
+
+Use the `injector.Injector` program to generate this data and publish to a
+PubSub topic. See the `Injector`javadocs for more information on how to run the
+injector. Set up the injector before you start one of these pipelines. Then,
+when you start the pipeline, pass as an argument the name of that PubSub topic.
+See the pipeline javadocs for the details.
+
+## Viewing the results in BigQuery
+
+All of the pipelines write their results to BigQuery.  `UserScore` and
+`HourlyTeamScore` each write one table, and `LeaderBoard` and
+`GameStats` each write two. The pipelines have default table names that
+you can override when you start up the pipeline if those tables already exist.
+
+Depending on the windowing intervals defined in a given pipeline, you may have
+to wait for a while (more than an hour) before you start to see results written
+to the BigQuery tables.


[20/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java
deleted file mode 100644
index 1ce5fe5..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java
+++ /dev/null
@@ -1,363 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.protobuf.ExtensionRegistry;
-import com.google.protobuf.Message;
-import com.google.protobuf.Parser;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Coder} using Google Protocol Buffers 2 binary format.
- *
- * <p>To learn more about Protocol Buffers, visit:
- * <a href="https://developers.google.com/protocol-buffers">https://developers.google.com/protocol-buffers</a>
- *
- * <p>To use, specify the {@link Coder} type on a PCollection containing Protocol Buffers messages.
- *
- * <pre>
- * {@code
- * PCollection<MyProto.Message> records =
- *     input.apply(...)
- *          .setCoder(Proto2Coder.of(MyProto.Message.class));
- * }
- * </pre>
- *
- * <p>Custom message extensions are also supported, but the coder must be made
- * aware of them explicitly:
- *
- * <pre>
- * {@code
- * PCollection<MyProto.Message> records =
- *     input.apply(...)
- *          .setCoder(Proto2Coder.of(MyProto.Message.class)
- *              .addExtensionsFrom(MyProto.class));
- * }
- * </pre>
- *
- * @param <T> the type of elements handled by this coder, must extend {@code Message}
- * @deprecated Use {@link ProtoCoder}.
- */
-@Deprecated
-public class Proto2Coder<T extends Message> extends AtomicCoder<T> {
-
-  /** The class of Protobuf message to be encoded. */
-  private final Class<T> protoMessageClass;
-
-  /**
-   * All extension host classes included in this Proto2Coder. The extensions from
-   * these classes will be included in the {@link ExtensionRegistry} used during
-   * encoding and decoding.
-   */
-  private final List<Class<?>> extensionHostClasses;
-
-  private Proto2Coder(Class<T> protoMessageClass, List<Class<?>> extensionHostClasses) {
-    this.protoMessageClass = protoMessageClass;
-    this.extensionHostClasses = extensionHostClasses;
-  }
-
-  private static final CoderProvider PROVIDER =
-      new CoderProvider() {
-        @Override
-        public <T> Coder<T> getCoder(TypeDescriptor<T> type) throws CannotProvideCoderException {
-          if (type.isSubtypeOf(new TypeDescriptor<Message>() {})) {
-            @SuppressWarnings("unchecked")
-            TypeDescriptor<? extends Message> messageType =
-                (TypeDescriptor<? extends Message>) type;
-            @SuppressWarnings("unchecked")
-            Coder<T> coder = (Coder<T>) Proto2Coder.of(messageType);
-            return coder;
-          } else {
-            throw new CannotProvideCoderException(
-                String.format(
-                    "Cannot provide Proto2Coder because %s "
-                        + "is not a subclass of protocol buffer Messsage",
-                    type));
-          }
-        }
-      };
-
-  public static CoderProvider coderProvider() {
-    return PROVIDER;
-  }
-
-  /**
-   * Returns a {@code Proto2Coder} for the given Protobuf message class.
-   */
-  public static <T extends Message> Proto2Coder<T> of(Class<T> protoMessageClass) {
-    return new Proto2Coder<T>(protoMessageClass, Collections.<Class<?>>emptyList());
-  }
-
-  /**
-   * Returns a {@code Proto2Coder} for the given Protobuf message class.
-   */
-  public static <T extends Message> Proto2Coder<T> of(TypeDescriptor<T> protoMessageType) {
-    @SuppressWarnings("unchecked")
-    Class<T> protoMessageClass = (Class<T>) protoMessageType.getRawType();
-    return of(protoMessageClass);
-  }
-
-  /**
-   * Produces a {@code Proto2Coder} like this one, but with the extensions from
-   * the given classes registered.
-   *
-   * @param moreExtensionHosts an iterable of classes that define a static
-   *      method {@code registerAllExtensions(ExtensionRegistry)}
-   */
-  public Proto2Coder<T> withExtensionsFrom(Iterable<Class<?>> moreExtensionHosts) {
-    for (Class<?> extensionHost : moreExtensionHosts) {
-      // Attempt to access the required method, to make sure it's present.
-      try {
-        Method registerAllExtensions =
-            extensionHost.getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class);
-        checkArgument(
-            Modifier.isStatic(registerAllExtensions.getModifiers()),
-            "Method registerAllExtensions() must be static for use with Proto2Coder");
-      } catch (NoSuchMethodException | SecurityException e) {
-        throw new IllegalArgumentException(e);
-      }
-    }
-
-    return new Proto2Coder<T>(
-        protoMessageClass,
-        new ImmutableList.Builder<Class<?>>()
-            .addAll(extensionHostClasses)
-            .addAll(moreExtensionHosts)
-            .build());
-  }
-
-  /**
-   * See {@link #withExtensionsFrom(Iterable)}.
-   */
-  public Proto2Coder<T> withExtensionsFrom(Class<?>... extensionHosts) {
-    return withExtensionsFrom(ImmutableList.copyOf(extensionHosts));
-  }
-
-  /**
-   * Adds custom Protobuf extensions to the coder. Returns {@code this}
-   * for method chaining.
-   *
-   * @param extensionHosts must be a class that defines a static
-   *      method name {@code registerAllExtensions}
-   * @deprecated use {@link #withExtensionsFrom}
-   */
-  @Deprecated
-  public Proto2Coder<T> addExtensionsFrom(Class<?>... extensionHosts) {
-    return addExtensionsFrom(ImmutableList.copyOf(extensionHosts));
-  }
-
-  /**
-   * Adds custom Protobuf extensions to the coder. Returns {@code this}
-   * for method chaining.
-   *
-   * @param extensionHosts must be a class that defines a static
-   *      method name {@code registerAllExtensions}
-   * @deprecated use {@link #withExtensionsFrom}
-   */
-  @Deprecated
-  public Proto2Coder<T> addExtensionsFrom(Iterable<Class<?>> extensionHosts) {
-    for (Class<?> extensionHost : extensionHosts) {
-      try {
-        // Attempt to access the declared method, to make sure it's present.
-        extensionHost.getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class);
-      } catch (NoSuchMethodException e) {
-        throw new IllegalArgumentException(e);
-      }
-      extensionHostClasses.add(extensionHost);
-    }
-    // The memoized extension registry needs to be recomputed because we have mutated this object.
-    synchronized (this) {
-      memoizedExtensionRegistry = null;
-      getExtensionRegistry();
-    }
-    return this;
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context) throws IOException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null " + protoMessageClass.getSimpleName());
-    }
-    if (context.isWholeStream) {
-      value.writeTo(outStream);
-    } else {
-      value.writeDelimitedTo(outStream);
-    }
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws IOException {
-    if (context.isWholeStream) {
-      return getParser().parseFrom(inStream, getExtensionRegistry());
-    } else {
-      return getParser().parseDelimitedFrom(inStream, getExtensionRegistry());
-    }
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (!(other instanceof Proto2Coder)) {
-      return false;
-    }
-    Proto2Coder<?> otherCoder = (Proto2Coder<?>) other;
-    return protoMessageClass.equals(otherCoder.protoMessageClass)
-        && Sets.newHashSet(extensionHostClasses)
-            .equals(Sets.newHashSet(otherCoder.extensionHostClasses));
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(protoMessageClass, extensionHostClasses);
-  }
-
-  /**
-   * The encoding identifier is designed to support evolution as per the design of Protocol
-   * Buffers. In order to use this class effectively, carefully follow the advice in the Protocol
-   * Buffers documentation at
-   * <a href="https://developers.google.com/protocol-buffers/docs/proto#updating">Updating
-   * A Message Type</a>.
-   *
-   * <p>In particular, the encoding identifier is guaranteed to be the same for {@code Proto2Coder}
-   * instances of the same principal message class, and otherwise distinct. Loaded extensions do not
-   * affect the id, nor does it encode the full schema.
-   *
-   * <p>When modifying a message class, here are the broadest guidelines; see the above link
-   * for greater detail.
-   *
-   * <ul>
-   * <li>Do not change the numeric tags for any fields.
-   * <li>Never remove a <code>required</code> field.
-   * <li>Only add <code>optional</code> or <code>repeated</code> fields, with sensible defaults.
-   * <li>When changing the type of a field, consult the Protocol Buffers documentation to ensure
-   * the new and old types are interchangeable.
-   * </ul>
-   *
-   * <p>Code consuming this message class should be prepared to support <i>all</i> versions of
-   * the class until it is certain that no remaining serialized instances exist.
-   *
-   * <p>If backwards incompatible changes must be made, the best recourse is to change the name
-   * of your Protocol Buffers message class.
-   */
-  @Override
-  public String getEncodingId() {
-    return protoMessageClass.getName();
-  }
-
-  private transient Parser<T> memoizedParser;
-
-  private Parser<T> getParser() {
-    if (memoizedParser == null) {
-      try {
-        @SuppressWarnings("unchecked")
-        T protoMessageInstance = (T) protoMessageClass.getMethod("getDefaultInstance").invoke(null);
-        @SuppressWarnings("unchecked")
-        Parser<T> tParser = (Parser<T>) protoMessageInstance.getParserForType();
-        memoizedParser = tParser;
-      } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
-        throw new IllegalArgumentException(e);
-      }
-    }
-    return memoizedParser;
-  }
-
-  private transient ExtensionRegistry memoizedExtensionRegistry;
-
-  private synchronized ExtensionRegistry getExtensionRegistry() {
-    if (memoizedExtensionRegistry == null) {
-      ExtensionRegistry registry = ExtensionRegistry.newInstance();
-      for (Class<?> extensionHost : extensionHostClasses) {
-        try {
-          extensionHost
-              .getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class)
-              .invoke(null, registry);
-        } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
-          throw new IllegalStateException(e);
-        }
-      }
-      memoizedExtensionRegistry = registry.getUnmodifiable();
-    }
-    return memoizedExtensionRegistry;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////
-  // JSON Serialization details below
-
-  private static final String PROTO_MESSAGE_CLASS = "proto_message_class";
-  private static final String PROTO_EXTENSION_HOSTS = "proto_extension_hosts";
-
-  /**
-   * Constructor for JSON deserialization only.
-   */
-  @JsonCreator
-  public static <T extends Message> Proto2Coder<T> of(
-      @JsonProperty(PROTO_MESSAGE_CLASS) String protoMessageClassName,
-      @Nullable @JsonProperty(PROTO_EXTENSION_HOSTS) List<String> extensionHostClassNames) {
-
-    try {
-      @SuppressWarnings("unchecked")
-      Class<T> protoMessageClass = (Class<T>) Class.forName(protoMessageClassName);
-      List<Class<?>> extensionHostClasses = Lists.newArrayList();
-      if (extensionHostClassNames != null) {
-        for (String extensionHostClassName : extensionHostClassNames) {
-          extensionHostClasses.add(Class.forName(extensionHostClassName));
-        }
-      }
-      return of(protoMessageClass).withExtensionsFrom(extensionHostClasses);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    Structs.addString(result, PROTO_MESSAGE_CLASS, protoMessageClass.getName());
-    List<CloudObject> extensionHostClassNames = Lists.newArrayList();
-    for (Class<?> clazz : extensionHostClasses) {
-      extensionHostClassNames.add(CloudObject.forString(clazz.getName()));
-    }
-    Structs.addList(result, PROTO_EXTENSION_HOSTS, extensionHostClassNames);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java
deleted file mode 100644
index 1590b11..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java
+++ /dev/null
@@ -1,184 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.ObjectStreamClass;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-/**
- * A {@link Coder} for Java classes that implement {@link Serializable}.
- *
- * <p>To use, specify the coder type on a PCollection:
- * <pre>
- * {@code
- *   PCollection<MyRecord> records =
- *       foo.apply(...).setCoder(SerializableCoder.of(MyRecord.class));
- * }
- * </pre>
- *
- * <p>{@link SerializableCoder} does not guarantee a deterministic encoding, as Java
- * serialization may produce different binary encodings for two equivalent
- * objects.
- *
- * @param <T> the type of elements handled by this coder
- */
-public class SerializableCoder<T extends Serializable> extends AtomicCoder<T> {
-
-  /**
-   * Returns a {@link SerializableCoder} instance for the provided element type.
-   * @param <T> the element type
-   */
-  public static <T extends Serializable> SerializableCoder<T> of(TypeDescriptor<T> type) {
-    @SuppressWarnings("unchecked")
-    Class<T> clazz = (Class<T>) type.getRawType();
-    return of(clazz);
-  }
-
-  /**
-   * Returns a {@link SerializableCoder} instance for the provided element class.
-   * @param <T> the element type
-   */
-  public static <T extends Serializable> SerializableCoder<T> of(Class<T> clazz) {
-    return new SerializableCoder<>(clazz);
-  }
-
-  @JsonCreator
-  @SuppressWarnings("unchecked")
-  public static SerializableCoder<?> of(@JsonProperty("type") String classType)
-      throws ClassNotFoundException {
-    Class<?> clazz = Class.forName(classType);
-    if (!Serializable.class.isAssignableFrom(clazz)) {
-      throw new ClassNotFoundException(
-          "Class " + classType + " does not implement Serializable");
-    }
-    return of((Class<? extends Serializable>) clazz);
-  }
-
-  /**
-   * A {@link CoderProvider} that constructs a {@link SerializableCoder}
-   * for any class that implements serializable.
-   */
-  public static final CoderProvider PROVIDER = new CoderProvider() {
-    @Override
-    public <T> Coder<T> getCoder(TypeDescriptor<T> typeDescriptor)
-        throws CannotProvideCoderException {
-      Class<?> clazz = typeDescriptor.getRawType();
-      if (Serializable.class.isAssignableFrom(clazz)) {
-        @SuppressWarnings("unchecked")
-        Class<? extends Serializable> serializableClazz =
-            (Class<? extends Serializable>) clazz;
-        @SuppressWarnings("unchecked")
-        Coder<T> coder = (Coder<T>) SerializableCoder.of(serializableClazz);
-        return coder;
-      } else {
-        throw new CannotProvideCoderException(
-            "Cannot provide SerializableCoder because " + typeDescriptor
-            + " does not implement Serializable");
-      }
-    }
-  };
-
-
-  private final Class<T> type;
-
-  protected SerializableCoder(Class<T> type) {
-    this.type = type;
-  }
-
-  public Class<T> getRecordType() {
-    return type;
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    try {
-      ObjectOutputStream oos = new ObjectOutputStream(outStream);
-      oos.writeObject(value);
-      oos.flush();
-    } catch (IOException exn) {
-      throw new CoderException("unable to serialize record " + value, exn);
-    }
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      ObjectInputStream ois = new ObjectInputStream(inStream);
-      return type.cast(ois.readObject());
-    } catch (ClassNotFoundException e) {
-      throw new CoderException("unable to deserialize record", e);
-    }
-  }
-
-  @Override
-  public String getEncodingId() {
-    return String.format("%s:%s",
-        type.getName(),
-        ObjectStreamClass.lookup(type).getSerialVersionUID());
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    result.put("type", type.getName());
-    return result;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always. Java serialization is not
-   *         deterministic with respect to {@link Object#equals} for all types.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Java Serialization may be non-deterministic.");
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (getClass() != other.getClass()) {
-      return false;
-    }
-    return type == ((SerializableCoder<?>) other).type;
-  }
-
-  @Override
-  public int hashCode() {
-    return type.hashCode();
-  }
-
-  // This coder inherits isRegisterByteSizeObserverCheap,
-  // getEncodedElementByteSize and registerByteSizeObserver
-  // from StandardCoder. Looks like we cannot do much better
-  // in this case.
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java
deleted file mode 100644
index 589a372..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java
+++ /dev/null
@@ -1,95 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * A {@link SetCoder} encodes any {@link Set} using the format of {@link IterableLikeCoder}. The
- * elements may not be in a deterministic order, depending on the {@code Set} implementation.
- *
- * @param <T> the type of the elements of the set
- */
-public class SetCoder<T> extends IterableLikeCoder<T, Set<T>> {
-
-  /**
-   * Produces a {@link SetCoder} with the given {@code elementCoder}.
-   */
-  public static <T> SetCoder<T> of(Coder<T> elementCoder) {
-    return new SetCoder<>(elementCoder);
-  }
-
-  /**
-   * Dynamically typed constructor for JSON deserialization.
-   */
-  @JsonCreator
-  public static SetCoder<?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Object> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
-    return of((Coder<?>) components.get(0));
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always. Sets are not ordered, but
-   *         they are encoded in the order of an arbitrary iteration.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "Ordering of elements in a set may be non-deterministic.");
-  }
-
-  /**
-   * Returns the first element in this set if it is non-empty,
-   * otherwise returns {@code null}.
-   */
-  public static <T> List<Object> getInstanceComponents(
-      Set<T> exampleValue) {
-    return getInstanceComponentsHelper(exampleValue);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Internal operations below here.
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return A new {@link Set} built from the elements in the {@link List} decoded by
-   * {@link IterableLikeCoder}.
-   */
-  @Override
-  protected final Set<T> decodeToIterable(List<T> decodedElements) {
-    return new HashSet<>(decodedElements);
-  }
-
-  protected SetCoder(Coder<T> elemCoder) {
-    super(elemCoder, "Set");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java
deleted file mode 100644
index ca189b1..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java
+++ /dev/null
@@ -1,230 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.common.collect.Lists;
-import com.google.common.io.ByteStreams;
-import com.google.common.io.CountingOutputStream;
-
-import java.io.ByteArrayOutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * An abstract base class to implement a {@link Coder} that defines equality, hashing, and printing
- * via the class name and recursively using {@link #getComponents}.
- *
- * <p>To extend {@link StandardCoder}, override the following methods as appropriate:
- *
- * <ul>
- *   <li>{@link #getComponents}: the default implementation returns {@link #getCoderArguments}.</li>
- *   <li>{@link #getEncodedElementByteSize} and
- *       {@link #isRegisterByteSizeObserverCheap}: the
- *       default implementation encodes values to bytes and counts the bytes, which is considered
- *       expensive.</li>
- *   <li>{@link #getEncodingId} and {@link #getAllowedEncodings}: by default, the encoding id
- *       is the empty string, so only the canonical name of the subclass will be used for
- *       compatibility checks, and no other encoding ids are allowed.</li>
- * </ul>
- */
-public abstract class StandardCoder<T> implements Coder<T> {
-  protected StandardCoder() {}
-
-  @Override
-  public String getEncodingId() {
-    return "";
-  }
-
-  @Override
-  public Collection<String> getAllowedEncodings() {
-    return Collections.emptyList();
-  }
-
-  /**
-   * Returns the list of {@link Coder Coders} that are components of this {@link Coder}.
-   */
-  public List<? extends Coder<?>> getComponents() {
-    List<? extends Coder<?>> coderArguments = getCoderArguments();
-    if (coderArguments == null) {
-      return Collections.emptyList();
-    } else {
-      return coderArguments;
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true} if the two {@link StandardCoder} instances have the
-   * same class and equal components.
-   */
-  @Override
-  public boolean equals(Object o) {
-    if (o == null || this.getClass() != o.getClass()) {
-      return false;
-    }
-    StandardCoder<?> that = (StandardCoder<?>) o;
-    return this.getComponents().equals(that.getComponents());
-  }
-
-  @Override
-  public int hashCode() {
-    return getClass().hashCode() * 31 + getComponents().hashCode();
-  }
-
-  @Override
-  public String toString() {
-    String s = getClass().getName();
-    s = s.substring(s.lastIndexOf('.') + 1);
-    List<? extends Coder<?>> componentCoders = getComponents();
-    if (!componentCoders.isEmpty()) {
-      s += "(";
-      boolean first = true;
-      for (Coder<?> componentCoder : componentCoders) {
-        if (first) {
-          first = false;
-        } else {
-          s += ", ";
-        }
-        s += componentCoder.toString();
-      }
-      s += ")";
-    }
-    return s;
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = CloudObject.forClass(getClass());
-
-    List<? extends Coder<?>> components = getComponents();
-    if (!components.isEmpty()) {
-      List<CloudObject> cloudComponents = new ArrayList<>(components.size());
-      for (Coder<?> coder : components) {
-        cloudComponents.add(coder.asCloudObject());
-      }
-      addList(result, PropertyNames.COMPONENT_ENCODINGS, cloudComponents);
-    }
-
-    String encodingId = getEncodingId();
-    checkNotNull(encodingId, "Coder.getEncodingId() must not return null.");
-    if (!encodingId.isEmpty()) {
-      addString(result, PropertyNames.ENCODING_ID, encodingId);
-    }
-
-    Collection<String> allowedEncodings = getAllowedEncodings();
-    if (!allowedEncodings.isEmpty()) {
-      addStringList(result, PropertyNames.ALLOWED_ENCODINGS, Lists.newArrayList(allowedEncodings));
-    }
-
-    return result;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code false} unless it is overridden. {@link StandardCoder#registerByteSizeObserver}
-   *         invokes {@link #getEncodedElementByteSize} which requires re-encoding an element
-   *         unless it is overridden. This is considered expensive.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(T value, Context context) {
-    return false;
-  }
-
-  /**
-   * Returns the size in bytes of the encoded value using this coder.
-   */
-  protected long getEncodedElementByteSize(T value, Context context)
-      throws Exception {
-    try {
-      CountingOutputStream os = new CountingOutputStream(ByteStreams.nullOutputStream());
-      encode(value, os, context);
-      return os.getCount();
-    } catch (Exception exn) {
-      throw new IllegalArgumentException(
-          "Unable to encode element '" + value + "' with coder '" + this + "'.", exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>For {@link StandardCoder} subclasses, this notifies {@code observer} about the byte size
-   * of the encoded value using this coder as returned by {@link #getEncodedElementByteSize}.
-   */
-  @Override
-  public void registerByteSizeObserver(
-      T value, ElementByteSizeObserver observer, Context context)
-      throws Exception {
-    observer.update(getEncodedElementByteSize(value, context));
-  }
-
-  protected void verifyDeterministic(String message, Iterable<Coder<?>> coders)
-      throws NonDeterministicException {
-    for (Coder<?> coder : coders) {
-      try {
-        coder.verifyDeterministic();
-      } catch (NonDeterministicException e) {
-        throw new NonDeterministicException(this, message, e);
-      }
-    }
-  }
-
-  protected void verifyDeterministic(String message, Coder<?>... coders)
-      throws NonDeterministicException {
-    verifyDeterministic(message, Arrays.asList(coders));
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code false} for {@link StandardCoder} unless overridden.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return false;
-  }
-
-  @Override
-  public Object structuralValue(T value) throws Exception {
-    if (value != null && consistentWithEquals()) {
-      return value;
-    } else {
-      try {
-        ByteArrayOutputStream os = new ByteArrayOutputStream();
-        encode(value, os, Context.OUTER);
-        return new StructuralByteArray(os.toByteArray());
-      } catch (Exception exn) {
-        throw new IllegalArgumentException(
-            "Unable to encode element '" + value + "' with coder '" + this + "'.", exn);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java
deleted file mode 100644
index 3f352d3..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java
+++ /dev/null
@@ -1,87 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * A {@link Coder} that wraps a {@code Coder<String>}
- * and encodes/decodes values via string representations.
- *
- * <p>To decode, the input byte stream is decoded to
- * a {@link String}, and this is passed to the single-argument
- * constructor for {@code T}.
- *
- * <p>To encode, the input value is converted via {@code toString()},
- * and this string is encoded.
- *
- * <p>In order for this to operate correctly for a class {@code Clazz},
- * it must be the case for any instance {@code x} that
- * {@code x.equals(new Clazz(x.toString()))}.
- *
- * <p>This method of encoding is not designed for ease of evolution of {@code Clazz};
- * it should only be used in cases where the class is stable or the encoding is not
- * important. If evolution of the class is important, see {@link ProtoCoder}, {@link AvroCoder},
- * or {@link JAXBCoder}.
- *
- * @param <T> The type of objects coded.
- */
-public class StringDelegateCoder<T> extends DelegateCoder<T, String> {
-  public static <T> StringDelegateCoder<T> of(Class<T> clazz) {
-    return new StringDelegateCoder<T>(clazz);
-  }
-
-  @Override
-  public String toString() {
-    return "StringDelegateCoder(" + clazz + ")";
-  }
-
-  private final Class<T> clazz;
-
-  protected StringDelegateCoder(final Class<T> clazz) {
-    super(StringUtf8Coder.of(),
-      new CodingFunction<T, String>() {
-        @Override
-        public String apply(T input) {
-          return input.toString();
-        }
-      },
-      new CodingFunction<String, T>() {
-        @Override
-        public T apply(String input) throws
-            NoSuchMethodException,
-            InstantiationException,
-            IllegalAccessException,
-            InvocationTargetException {
-          return clazz.getConstructor(String.class).newInstance(input);
-        }
-      });
-
-    this.clazz = clazz;
-  }
-
-  /**
-   * The encoding id is the fully qualified name of the encoded/decoded class.
-   */
-  @Override
-  public String getEncodingId() {
-    return clazz.getName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java
deleted file mode 100644
index 25b8f5e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java
+++ /dev/null
@@ -1,140 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.ExposedByteArrayOutputStream;
-import com.google.cloud.dataflow.sdk.util.StreamUtils;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.common.base.Utf8;
-import com.google.common.io.ByteStreams;
-import com.google.common.io.CountingOutputStream;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-import java.nio.charset.StandardCharsets;
-
-/**
- * A {@link Coder} that encodes {@link String Strings} in UTF-8 encoding.
- * If in a nested context, prefixes the string with an integer length field,
- * encoded via a {@link VarIntCoder}.
- */
-public class StringUtf8Coder extends AtomicCoder<String> {
-
-  @JsonCreator
-  public static StringUtf8Coder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final StringUtf8Coder INSTANCE = new StringUtf8Coder();
-
-  private static void writeString(String value, DataOutputStream dos)
-      throws IOException {
-    byte[] bytes = value.getBytes(StandardCharsets.UTF_8);
-    VarInt.encode(bytes.length, dos);
-    dos.write(bytes);
-  }
-
-  private static String readString(DataInputStream dis) throws IOException {
-    int len = VarInt.decodeInt(dis);
-    if (len < 0) {
-      throw new CoderException("Invalid encoded string length: " + len);
-    }
-    byte[] bytes = new byte[len];
-    dis.readFully(bytes);
-    return new String(bytes, StandardCharsets.UTF_8);
-  }
-
-  private StringUtf8Coder() {}
-
-  @Override
-  public void encode(String value, OutputStream outStream, Context context)
-      throws IOException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null String");
-    }
-    if (context.isWholeStream) {
-      byte[] bytes = value.getBytes(StandardCharsets.UTF_8);
-      if (outStream instanceof ExposedByteArrayOutputStream) {
-        ((ExposedByteArrayOutputStream) outStream).writeAndOwn(bytes);
-      } else {
-        outStream.write(bytes);
-      }
-    } else {
-      writeString(value, new DataOutputStream(outStream));
-    }
-  }
-
-  @Override
-  public String decode(InputStream inStream, Context context)
-      throws IOException {
-    if (context.isWholeStream) {
-      byte[] bytes = StreamUtils.getBytes(inStream);
-      return new String(bytes, StandardCharsets.UTF_8);
-    } else {
-      try {
-        return readString(new DataInputStream(inStream));
-      } catch (EOFException | UTFDataFormatException exn) {
-        // These exceptions correspond to decoding problems, so change
-        // what kind of exception they're branded as.
-        throw new CoderException(exn);
-      }
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return the byte size of the UTF-8 encoding of the a string or, in a nested context,
-   * the byte size of the encoding plus the encoded length prefix.
-   */
-  @Override
-  protected long getEncodedElementByteSize(String value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null String");
-    }
-    if (context.isWholeStream) {
-      return Utf8.encodedLength(value);
-    } else {
-      CountingOutputStream countingStream =
-          new CountingOutputStream(ByteStreams.nullOutputStream());
-      DataOutputStream stream = new DataOutputStream(countingStream);
-      writeString(value, stream);
-      return countingStream.getCount();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StructuralByteArray.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StructuralByteArray.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StructuralByteArray.java
deleted file mode 100644
index aa44456..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/StructuralByteArray.java
+++ /dev/null
@@ -1,58 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.api.client.util.Base64.encodeBase64String;
-
-import java.util.Arrays;
-
-/**
- * A wrapper around a byte[] that uses structural, value-based
- * equality rather than byte[]'s normal object identity.
- */
-public class StructuralByteArray {
-  byte[] value;
-
-  public StructuralByteArray(byte[] value) {
-    this.value = value;
-  }
-
-  public byte[] getValue() {
-    return value;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof StructuralByteArray) {
-      StructuralByteArray that = (StructuralByteArray) o;
-      return Arrays.equals(this.value, that.value);
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return Arrays.hashCode(value);
-  }
-
-  @Override
-  public String toString() {
-    return "base64:" + encodeBase64String(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java
deleted file mode 100644
index b02fb08..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java
+++ /dev/null
@@ -1,83 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.api.services.bigquery.model.TableRow;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} that encodes BigQuery {@link TableRow} objects in their native JSON format.
- */
-public class TableRowJsonCoder extends AtomicCoder<TableRow> {
-
-  @JsonCreator
-  public static TableRowJsonCoder of() {
-    return INSTANCE;
-  }
-
-  @Override
-  public void encode(TableRow value, OutputStream outStream, Context context)
-      throws IOException {
-    String strValue = MAPPER.writeValueAsString(value);
-    StringUtf8Coder.of().encode(strValue, outStream, context);
-  }
-
-  @Override
-  public TableRow decode(InputStream inStream, Context context)
-      throws IOException {
-    String strValue = StringUtf8Coder.of().decode(inStream, context);
-    return MAPPER.readValue(strValue, TableRow.class);
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(TableRow value, Context context)
-      throws Exception {
-    String strValue = MAPPER.writeValueAsString(value);
-    return StringUtf8Coder.of().getEncodedElementByteSize(strValue, context);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  // FAIL_ON_EMPTY_BEANS is disabled in order to handle null values in
-  // TableRow.
-  private static final ObjectMapper MAPPER =
-      new ObjectMapper().disable(SerializationFeature.FAIL_ON_EMPTY_BEANS);
-
-  private static final TableRowJsonCoder INSTANCE = new TableRowJsonCoder();
-
-  private TableRowJsonCoder() { }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException always. A {@link TableRow} can hold arbitrary
-   *         {@link Object} instances, which makes the encoding non-deterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    throw new NonDeterministicException(this,
-        "TableCell can hold arbitrary instances, which may be non-deterministic.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java
deleted file mode 100644
index 539c56a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java
+++ /dev/null
@@ -1,70 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} that encodes {@code Integer Integers} as the ASCII bytes of
- * their textual, decimal, representation.
- */
-public class TextualIntegerCoder extends AtomicCoder<Integer> {
-
-  @JsonCreator
-  public static TextualIntegerCoder of() {
-    return new TextualIntegerCoder();
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  protected TextualIntegerCoder() {}
-
-  @Override
-  public void encode(Integer value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    String textualValue = value.toString();
-    StringUtf8Coder.of().encode(textualValue, outStream, context);
-  }
-
-  @Override
-  public Integer decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    String textualValue = StringUtf8Coder.of().decode(inStream, context);
-    try {
-      return Integer.valueOf(textualValue);
-    } catch (NumberFormatException exn) {
-      throw new CoderException("error when decoding a textual integer", exn);
-    }
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(Integer value, Context context) throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    String textualValue = value.toString();
-    return StringUtf8Coder.of().getEncodedElementByteSize(textualValue, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java
deleted file mode 100644
index 42862bb..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java
+++ /dev/null
@@ -1,98 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.VarInt;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link Coder} that encodes {@link Integer Integers} using between 1 and 5 bytes. Negative
- * numbers always take 5 bytes, so {@link BigEndianIntegerCoder} may be preferable for
- * integers that are known to often be large or negative.
- */
-public class VarIntCoder extends AtomicCoder<Integer> {
-
-  @JsonCreator
-  public static VarIntCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final VarIntCoder INSTANCE =
-      new VarIntCoder();
-
-  private VarIntCoder() {}
-
-  @Override
-  public void encode(Integer value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    VarInt.encode(value.intValue(), outStream);
-  }
-
-  @Override
-  public Integer decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      return VarInt.decodeInt(inStream);
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link VarIntCoder} is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link #getEncodedElementByteSize} is cheap.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Integer value, Context context) {
-    return true;
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(Integer value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Integer");
-    }
-    return VarInt.getLength(value.longValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java
deleted file mode 100644
index 669453e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java
+++ /dev/null
@@ -1,97 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.VarInt;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link Coder} that encodes {@link Long Longs} using between 1 and 10 bytes. Negative
- * numbers always take 10 bytes, so {@link BigEndianLongCoder} may be preferable for
- * longs that are known to often be large or negative.
- */
-public class VarLongCoder extends AtomicCoder<Long> {
-
-  @JsonCreator
-  public static VarLongCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final VarLongCoder INSTANCE = new VarLongCoder();
-
-  private VarLongCoder() {}
-
-  @Override
-  public void encode(Long value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Long");
-    }
-    VarInt.encode(value.longValue(), outStream);
-  }
-
-  @Override
-  public Long decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      return VarInt.decodeLong(inStream);
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link VarLongCoder} is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link #getEncodedElementByteSize} is cheap.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Long value, Context context) {
-    return true;
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(Long value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Long");
-    }
-    return VarInt.getLength(value.longValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java
deleted file mode 100644
index 813ee2f..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java
+++ /dev/null
@@ -1,77 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} for {@link Void}. Uses zero bytes per {@link Void}.
- */
-public class VoidCoder extends AtomicCoder<Void> {
-
-  @JsonCreator
-  public static VoidCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final VoidCoder INSTANCE = new VoidCoder();
-
-  private VoidCoder() {}
-
-  @Override
-  public void encode(Void value, OutputStream outStream, Context context) {
-    // Nothing to write!
-  }
-
-  @Override
-  public Void decode(InputStream inStream, Context context) {
-    // Nothing to read!
-    return null;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return  {@code true}. {@link VoidCoder} is (vacuously) injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link VoidCoder#getEncodedElementByteSize} runs in constant time.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Void value, Context context) {
-    return true;
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(Void value, Context context)
-      throws Exception {
-    return 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java
deleted file mode 100644
index a3bc150..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java
+++ /dev/null
@@ -1,45 +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.
- */
-/**
- * Defines {@link com.google.cloud.dataflow.sdk.coders.Coder Coders}
- * to specify how data is encoded to and decoded from byte strings.
- *
- * <p>During execution of a Pipeline, elements in a
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}
- * may need to be encoded into byte strings.
- * This happens both at the beginning and end of a pipeline when data is read from and written to
- * persistent storage and also during execution of a pipeline when elements are communicated between
- * machines.
- *
- * <p>Exactly when PCollection elements are encoded during execution depends on which
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is being used and how that runner
- * chooses to execute the pipeline. As such, Dataflow requires that all PCollections have an
- * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from
- * the available Java type
- * information and the Pipeline's {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry}. It
- * can be specified per PCollection via
- * {@link com.google.cloud.dataflow.sdk.values.PCollection#setCoder(Coder)} or per type using the
- * {@link com.google.cloud.dataflow.sdk.coders.DefaultCoder} annotation.
- *
- * <p>This package provides a number of coders for common types like {@code Integer},
- * {@code String}, and {@code List}, as well as coders like
- * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} that can be used to encode many custom
- * types.
- *
- */
-package com.google.cloud.dataflow.sdk.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtoCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtoCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtoCoder.java
deleted file mode 100644
index 88d13e9..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtoCoder.java
+++ /dev/null
@@ -1,406 +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 com.google.cloud.dataflow.sdk.coders.protobuf;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderProvider;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.protobuf.ExtensionRegistry;
-import com.google.protobuf.Message;
-import com.google.protobuf.Parser;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Coder} using Google Protocol Buffers binary format. {@link ProtoCoder} supports both
- * Protocol Buffers syntax versions 2 and 3.
- *
- * <p>To learn more about Protocol Buffers, visit:
- * <a href="https://developers.google.com/protocol-buffers">https://developers.google.com/protocol-buffers</a>
- *
- * <p>{@link ProtoCoder} is registered in the global {@link CoderRegistry} as the default
- * {@link Coder} for any {@link Message} object. Custom message extensions are also supported, but
- * these extensions must be registered for a particular {@link ProtoCoder} instance and that
- * instance must be registered on the {@link PCollection} that needs the extensions:
- *
- * <pre>{@code
- * import MyProtoFile;
- * import MyProtoFile.MyMessage;
- *
- * Coder<MyMessage> coder = ProtoCoder.of(MyMessage.class).withExtensionsFrom(MyProtoFile.class);
- * PCollection<MyMessage> records =  input.apply(...).setCoder(coder);
- * }</pre>
- *
- * <h3>Versioning</h3>
- *
- * <p>{@link ProtoCoder} supports both versions 2 and 3 of the Protocol Buffers syntax. However,
- * the Java runtime version of the <code>google.com.protobuf</code> library must match exactly the
- * version of <code>protoc</code> that was used to produce the JAR files containing the compiled
- * <code>.proto</code> messages.
- *
- * <p>For more information, see the
- * <a href="https://developers.google.com/protocol-buffers/docs/proto3#using-proto2-message-types">Protocol Buffers documentation</a>.
- *
- * <h3>{@link ProtoCoder} and Determinism</h3>
- *
- * <p>In general, Protocol Buffers messages can be encoded deterministically within a single
- * pipeline as long as:
- *
- * <ul>
- * <li>The encoded messages (and any transitively linked messages) do not use <code>map</code>
- *     fields.</li>
- * <li>Every Java VM that encodes or decodes the messages use the same runtime version of the
- *     Protocol Buffers library and the same compiled <code>.proto</code> file JAR.</li>
- * </ul>
- *
- * <h3>{@link ProtoCoder} and Encoding Stability</h3>
- *
- * <p>When changing Protocol Buffers messages, follow the rules in the Protocol Buffers language
- * guides for
- * <a href="https://developers.google.com/protocol-buffers/docs/proto#updating">{@code proto2}</a>
- * and
- * <a href="https://developers.google.com/protocol-buffers/docs/proto3#updating">{@code proto3}</a>
- * syntaxes, depending on your message type. Following these guidelines will ensure that the
- * old encoded data can be read by new versions of the code.
- *
- * <p>Generally, any change to the message type, registered extensions, runtime library, or
- * compiled proto JARs may change the encoding. Thus even if both the original and updated messages
- * can be encoded deterministically within a single job, these deterministic encodings may not be
- * the same across jobs.
- *
- * @param <T> the Protocol Buffers {@link Message} handled by this {@link Coder}.
- */
-public class ProtoCoder<T extends Message> extends AtomicCoder<T> {
-
-  /**
-   * A {@link CoderProvider} that returns a {@link ProtoCoder} with an empty
-   * {@link ExtensionRegistry}.
-   */
-  public static CoderProvider coderProvider() {
-    return PROVIDER;
-  }
-
-  /**
-   * Returns a {@link ProtoCoder} for the given Protocol Buffers {@link Message}.
-   */
-  public static <T extends Message> ProtoCoder<T> of(Class<T> protoMessageClass) {
-    return new ProtoCoder<T>(protoMessageClass, ImmutableSet.<Class<?>>of());
-  }
-
-  /**
-   * Returns a {@link ProtoCoder} for the Protocol Buffers {@link Message} indicated by the given
-   * {@link TypeDescriptor}.
-   */
-  public static <T extends Message> ProtoCoder<T> of(TypeDescriptor<T> protoMessageType) {
-    @SuppressWarnings("unchecked")
-    Class<T> protoMessageClass = (Class<T>) protoMessageType.getRawType();
-    return of(protoMessageClass);
-  }
-
-  /**
-   * Returns a {@link ProtoCoder} like this one, but with the extensions from the given classes
-   * registered.
-   *
-   * <p>Each of the extension host classes must be an class automatically generated by the
-   * Protocol Buffers compiler, {@code protoc}, that contains messages.
-   *
-   * <p>Does not modify this object.
-   */
-  public ProtoCoder<T> withExtensionsFrom(Iterable<Class<?>> moreExtensionHosts) {
-    for (Class<?> extensionHost : moreExtensionHosts) {
-      // Attempt to access the required method, to make sure it's present.
-      try {
-        Method registerAllExtensions =
-            extensionHost.getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class);
-        checkArgument(
-            Modifier.isStatic(registerAllExtensions.getModifiers()),
-            "Method registerAllExtensions() must be static");
-      } catch (NoSuchMethodException | SecurityException e) {
-        throw new IllegalArgumentException(
-            String.format("Unable to register extensions for %s", extensionHost.getCanonicalName()),
-            e);
-      }
-    }
-
-    return new ProtoCoder<T>(
-        protoMessageClass,
-        new ImmutableSet.Builder<Class<?>>()
-            .addAll(extensionHostClasses)
-            .addAll(moreExtensionHosts)
-            .build());
-  }
-
-  /**
-   * See {@link #withExtensionsFrom(Iterable)}.
-   *
-   * <p>Does not modify this object.
-   */
-  public ProtoCoder<T> withExtensionsFrom(Class<?>... moreExtensionHosts) {
-    return withExtensionsFrom(Arrays.asList(moreExtensionHosts));
-  }
-
-  @Override
-  public void encode(T value, OutputStream outStream, Context context) throws IOException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null " + protoMessageClass.getSimpleName());
-    }
-    if (context.isWholeStream) {
-      value.writeTo(outStream);
-    } else {
-      value.writeDelimitedTo(outStream);
-    }
-  }
-
-  @Override
-  public T decode(InputStream inStream, Context context) throws IOException {
-    if (context.isWholeStream) {
-      return getParser().parseFrom(inStream, getExtensionRegistry());
-    } else {
-      return getParser().parseDelimitedFrom(inStream, getExtensionRegistry());
-    }
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (!(other instanceof ProtoCoder)) {
-      return false;
-    }
-    ProtoCoder<?> otherCoder = (ProtoCoder<?>) other;
-    return protoMessageClass.equals(otherCoder.protoMessageClass)
-        && Sets.newHashSet(extensionHostClasses)
-            .equals(Sets.newHashSet(otherCoder.extensionHostClasses));
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(protoMessageClass, extensionHostClasses);
-  }
-
-  /**
-   * The encoding identifier is designed to support evolution as per the design of Protocol
-   * Buffers. In order to use this class effectively, carefully follow the advice in the Protocol
-   * Buffers documentation at
-   * <a href="https://developers.google.com/protocol-buffers/docs/proto#updating">Updating
-   * A Message Type</a>.
-   *
-   * <p>In particular, the encoding identifier is guaranteed to be the same for {@link ProtoCoder}
-   * instances of the same principal message class, with the same registered extension host classes,
-   * and otherwise distinct. Note that the encoding ID does not encode any version of the message
-   * or extensions, nor does it include the message schema.
-   *
-   * <p>When modifying a message class, here are the broadest guidelines; see the above link
-   * for greater detail.
-   *
-   * <ul>
-   * <li>Do not change the numeric tags for any fields.
-   * <li>Never remove a <code>required</code> field.
-   * <li>Only add <code>optional</code> or <code>repeated</code> fields, with sensible defaults.
-   * <li>When changing the type of a field, consult the Protocol Buffers documentation to ensure
-   * the new and old types are interchangeable.
-   * </ul>
-   *
-   * <p>Code consuming this message class should be prepared to support <i>all</i> versions of
-   * the class until it is certain that no remaining serialized instances exist.
-   *
-   * <p>If backwards incompatible changes must be made, the best recourse is to change the name
-   * of your Protocol Buffers message class.
-   */
-  @Override
-  public String getEncodingId() {
-    return protoMessageClass.getName() + getSortedExtensionClasses().toString();
-  }
-
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    ProtobufUtil.verifyDeterministic(this);
-  }
-
-  /**
-   * Returns the Protocol Buffers {@link Message} type this {@link ProtoCoder} supports.
-   */
-  public Class<T> getMessageType() {
-    return protoMessageClass;
-  }
-
-  /**
-   * Returns the {@link ExtensionRegistry} listing all known Protocol Buffers extension messages
-   * to {@code T} registered with this {@link ProtoCoder}.
-   */
-  public ExtensionRegistry getExtensionRegistry() {
-    if (memoizedExtensionRegistry == null) {
-      ExtensionRegistry registry = ExtensionRegistry.newInstance();
-      for (Class<?> extensionHost : extensionHostClasses) {
-        try {
-          extensionHost
-              .getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class)
-              .invoke(null, registry);
-        } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
-          throw new IllegalStateException(e);
-        }
-      }
-      memoizedExtensionRegistry = registry.getUnmodifiable();
-    }
-    return memoizedExtensionRegistry;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////
-  // Private implementation details below.
-
-  /** The {@link Message} type to be coded. */
-  private final Class<T> protoMessageClass;
-
-  /**
-   * All extension host classes included in this {@link ProtoCoder}. The extensions from these
-   * classes will be included in the {@link ExtensionRegistry} used during encoding and decoding.
-   */
-  private final Set<Class<?>> extensionHostClasses;
-
-  // Constants used to serialize and deserialize
-  private static final String PROTO_MESSAGE_CLASS = "proto_message_class";
-  private static final String PROTO_EXTENSION_HOSTS = "proto_extension_hosts";
-
-  // Transient fields that are lazy initialized and then memoized.
-  private transient ExtensionRegistry memoizedExtensionRegistry;
-  private transient Parser<T> memoizedParser;
-
-  /** Private constructor. */
-  private ProtoCoder(Class<T> protoMessageClass, Set<Class<?>> extensionHostClasses) {
-    this.protoMessageClass = protoMessageClass;
-    this.extensionHostClasses = extensionHostClasses;
-  }
-
-  /**
-   * @deprecated For JSON deserialization only.
-   */
-  @JsonCreator
-  @Deprecated
-  public static <T extends Message> ProtoCoder<T> of(
-      @JsonProperty(PROTO_MESSAGE_CLASS) String protoMessageClassName,
-      @Nullable @JsonProperty(PROTO_EXTENSION_HOSTS) List<String> extensionHostClassNames) {
-
-    try {
-      @SuppressWarnings("unchecked")
-      Class<T> protoMessageClass = (Class<T>) Class.forName(protoMessageClassName);
-      List<Class<?>> extensionHostClasses = Lists.newArrayList();
-      if (extensionHostClassNames != null) {
-        for (String extensionHostClassName : extensionHostClassNames) {
-          extensionHostClasses.add(Class.forName(extensionHostClassName));
-        }
-      }
-      return of(protoMessageClass).withExtensionsFrom(extensionHostClasses);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  @Override
-  public CloudObject asCloudObject() {
-    CloudObject result = super.asCloudObject();
-    Structs.addString(result, PROTO_MESSAGE_CLASS, protoMessageClass.getName());
-    List<CloudObject> extensionHostClassNames = Lists.newArrayList();
-    for (String className : getSortedExtensionClasses()) {
-      extensionHostClassNames.add(CloudObject.forString(className));
-    }
-    Structs.addList(result, PROTO_EXTENSION_HOSTS, extensionHostClassNames);
-    return result;
-  }
-
-  /** Get the memoized {@link Parser}, possibly initializing it lazily. */
-  private Parser<T> getParser() {
-    if (memoizedParser == null) {
-      try {
-        @SuppressWarnings("unchecked")
-        T protoMessageInstance = (T) protoMessageClass.getMethod("getDefaultInstance").invoke(null);
-        @SuppressWarnings("unchecked")
-        Parser<T> tParser = (Parser<T>) protoMessageInstance.getParserForType();
-        memoizedParser = tParser;
-      } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
-        throw new IllegalArgumentException(e);
-      }
-    }
-    return memoizedParser;
-  }
-
-  /**
-   * The implementation of the {@link CoderProvider} for this {@link ProtoCoder} returned by
-   * {@link #coderProvider()}.
-   */
-  private static final CoderProvider PROVIDER =
-      new CoderProvider() {
-        @Override
-        public <T> Coder<T> getCoder(TypeDescriptor<T> type) throws CannotProvideCoderException {
-          if (!type.isSubtypeOf(new TypeDescriptor<Message>() {})) {
-            throw new CannotProvideCoderException(
-                String.format(
-                    "Cannot provide %s because %s is not a subclass of %s",
-                    ProtoCoder.class.getSimpleName(),
-                    type,
-                    Message.class.getName()));
-          }
-
-          @SuppressWarnings("unchecked")
-          TypeDescriptor<? extends Message> messageType = (TypeDescriptor<? extends Message>) type;
-          try {
-            @SuppressWarnings("unchecked")
-            Coder<T> coder = (Coder<T>) ProtoCoder.of(messageType);
-            return coder;
-          } catch (IllegalArgumentException e) {
-            throw new CannotProvideCoderException(e);
-          }
-        }
-      };
-
-  private SortedSet<String> getSortedExtensionClasses() {
-    SortedSet<String> ret = new TreeSet<>();
-    for (Class<?> clazz : extensionHostClasses) {
-      ret.add(clazz.getName());
-    }
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtobufUtil.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtobufUtil.java
deleted file mode 100644
index 24415ac..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/protobuf/ProtobufUtil.java
+++ /dev/null
@@ -1,172 +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 com.google.cloud.dataflow.sdk.coders.protobuf;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.protobuf.Descriptors.Descriptor;
-import com.google.protobuf.Descriptors.FieldDescriptor;
-import com.google.protobuf.Descriptors.FileDescriptor.Syntax;
-import com.google.protobuf.Descriptors.GenericDescriptor;
-import com.google.protobuf.ExtensionRegistry;
-import com.google.protobuf.ExtensionRegistry.ExtensionInfo;
-import com.google.protobuf.Message;
-
-import java.lang.reflect.InvocationTargetException;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Utility functions for reflecting and analyzing Protocol Buffers classes.
- *
- * <p>Used by {@link ProtoCoder}, but in a separate file for testing and isolation.
- */
-class ProtobufUtil {
-  /**
-   * Returns the {@link Descriptor} for the given Protocol Buffers {@link Message}.
-   *
-   * @throws IllegalArgumentException if there is an error in Java reflection.
-   */
-  static Descriptor getDescriptorForClass(Class<? extends Message> clazz) {
-    try {
-      return (Descriptor) clazz.getMethod("getDescriptor").invoke(null);
-    } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  /**
-   * Returns the {@link Descriptor} for the given Protocol Buffers {@link Message} as well as
-   * every class it can include transitively.
-   *
-   * @throws IllegalArgumentException if there is an error in Java reflection.
-   */
-  static Set<Descriptor> getRecursiveDescriptorsForClass(
-      Class<? extends Message> clazz, ExtensionRegistry registry) {
-    Descriptor root = getDescriptorForClass(clazz);
-    Set<Descriptor> descriptors = new HashSet<>();
-    recursivelyAddDescriptors(root, descriptors, registry);
-    return descriptors;
-  }
-
-  /**
-   * Recursively walks the given {@link Message} class and verifies that every field or message
-   * linked in uses the Protocol Buffers proto2 syntax.
-   */
-  static void checkProto2Syntax(Class<? extends Message> clazz, ExtensionRegistry registry) {
-    for (GenericDescriptor d : getRecursiveDescriptorsForClass(clazz, registry)) {
-      Syntax s = d.getFile().getSyntax();
-      checkArgument(
-          s == Syntax.PROTO2,
-          "Message %s or one of its dependencies does not use proto2 syntax: %s in file %s",
-          clazz.getName(),
-          d.getFullName(),
-          d.getFile().getName());
-    }
-  }
-
-  /**
-   * Recursively checks whether the specified class uses any Protocol Buffers fields that cannot
-   * be deterministically encoded.
-   *
-   * @throws NonDeterministicException if the object cannot be encoded deterministically.
-   */
-  static void verifyDeterministic(ProtoCoder<?> coder) throws NonDeterministicException {
-    Class<? extends Message> message = coder.getMessageType();
-    ExtensionRegistry registry = coder.getExtensionRegistry();
-    Set<Descriptor> descriptors = getRecursiveDescriptorsForClass(message, registry);
-    for (Descriptor d : descriptors) {
-      for (FieldDescriptor fd : d.getFields()) {
-        // If there is a transitively reachable Protocol Buffers map field, then this object cannot
-        // be encoded deterministically.
-        if (fd.isMapField()) {
-          String reason =
-              String.format(
-                  "Protocol Buffers message %s transitively includes Map field %s (from file %s)."
-                      + " Maps cannot be deterministically encoded.",
-                  message.getName(),
-                  fd.getFullName(),
-                  fd.getFile().getFullName());
-          throw new NonDeterministicException(coder, reason);
-        }
-      }
-    }
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-  // Disable construction of utility class
-  private ProtobufUtil() {}
-
-  private static void recursivelyAddDescriptors(
-      Descriptor message, Set<Descriptor> descriptors, ExtensionRegistry registry) {
-    if (descriptors.contains(message)) {
-      return;
-    }
-    descriptors.add(message);
-
-    for (FieldDescriptor f : message.getFields()) {
-      recursivelyAddDescriptors(f, descriptors, registry);
-    }
-    for (FieldDescriptor f : message.getExtensions()) {
-      recursivelyAddDescriptors(f, descriptors, registry);
-    }
-    for (ExtensionInfo info :
-        registry.getAllImmutableExtensionsByExtendedType(message.getFullName())) {
-      recursivelyAddDescriptors(info.descriptor, descriptors, registry);
-    }
-    for (ExtensionInfo info :
-        registry.getAllMutableExtensionsByExtendedType(message.getFullName())) {
-      recursivelyAddDescriptors(info.descriptor, descriptors, registry);
-    }
-  }
-
-  private static void recursivelyAddDescriptors(
-      FieldDescriptor field, Set<Descriptor> descriptors, ExtensionRegistry registry) {
-    switch (field.getType()) {
-      case BOOL:
-      case BYTES:
-      case DOUBLE:
-      case ENUM:
-      case FIXED32:
-      case FIXED64:
-      case FLOAT:
-      case INT32:
-      case INT64:
-      case SFIXED32:
-      case SFIXED64:
-      case SINT32:
-      case SINT64:
-      case STRING:
-      case UINT32:
-      case UINT64:
-        // Primitive types do not transitively access anything else.
-        break;
-
-      case GROUP:
-      case MESSAGE:
-        // Recursively adds all the fields from this nested Message.
-        recursivelyAddDescriptors(field.getMessageType(), descriptors, registry);
-        break;
-
-      default:
-        throw new UnsupportedOperationException(
-            "Unexpected Protocol Buffers field type: " + field.getType());
-    }
-  }
-}



[47/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java
deleted file mode 100644
index f1d8d1a..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java
+++ /dev/null
@@ -1,224 +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 com.google.cloud.dataflow.examples.complete;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableComparator;
-import com.google.cloud.dataflow.sdk.transforms.Top;
-import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.util.List;
-
-/**
- * An example that reads Wikipedia edit data from Cloud Storage and computes the user with
- * the longest string of edits separated by no more than an hour within each month.
- *
- * <p>Concepts: Using Windowing to perform time-based aggregations of data.
- *
- * <p>It is not recommended to execute this pipeline locally, given the size of the default input
- * data.
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
- *
- * <p>The default input is {@code gs://dataflow-samples/wikipedia_edits/*.json} and can be
- * overridden with {@code --input}.
- *
- * <p>The input for this example is large enough that it's a good place to enable (experimental)
- * autoscaling:
- * <pre>{@code
- *   --autoscalingAlgorithm=BASIC
- *   --maxNumWorkers=20
- * }
- * </pre>
- * This will automatically scale the number of workers up over time until the job completes.
- */
-public class TopWikipediaSessions {
-  private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json";
-
-  /**
-   * Extracts user and timestamp from a TableRow representing a Wikipedia edit.
-   */
-  static class ExtractUserAndTimestamp extends DoFn<TableRow, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = c.element();
-      int timestamp = (Integer) row.get("timestamp");
-      String userName = (String) row.get("contributor_username");
-      if (userName != null) {
-        // Sets the implicit timestamp field to be used in windowing.
-        c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
-      }
-    }
-  }
-
-  /**
-   * Computes the number of edits in each user session.  A session is defined as
-   * a string of edits where each is separated from the next by less than an hour.
-   */
-  static class ComputeSessions
-      extends PTransform<PCollection<String>, PCollection<KV<String, Long>>> {
-    @Override
-    public PCollection<KV<String, Long>> apply(PCollection<String> actions) {
-      return actions
-          .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardHours(1))))
-
-          .apply(Count.<String>perElement());
-    }
-  }
-
-  /**
-   * Computes the longest session ending in each month.
-   */
-  private static class TopPerMonth
-      extends PTransform<PCollection<KV<String, Long>>, PCollection<List<KV<String, Long>>>> {
-    @Override
-    public PCollection<List<KV<String, Long>>> apply(PCollection<KV<String, Long>> sessions) {
-      return sessions
-        .apply(Window.<KV<String, Long>>into(CalendarWindows.months(1)))
-
-          .apply(Top.of(1, new SerializableComparator<KV<String, Long>>() {
-                    @Override
-                    public int compare(KV<String, Long> o1, KV<String, Long> o2) {
-                      return Long.compare(o1.getValue(), o2.getValue());
-                    }
-                  }).withoutDefaults());
-    }
-  }
-
-  static class SessionsToStringsDoFn extends DoFn<KV<String, Long>, KV<String, Long>>
-      implements RequiresWindowAccess {
-
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(KV.of(
-          c.element().getKey() + " : " + c.window(), c.element().getValue()));
-    }
-  }
-
-  static class FormatOutputDoFn extends DoFn<List<KV<String, Long>>, String>
-      implements RequiresWindowAccess {
-    @Override
-    public void processElement(ProcessContext c) {
-      for (KV<String, Long> item : c.element()) {
-        String session = item.getKey();
-        long count = item.getValue();
-        c.output(session + " : " + count + " : " + ((IntervalWindow) c.window()).start());
-      }
-    }
-  }
-
-  static class ComputeTopSessions extends PTransform<PCollection<TableRow>, PCollection<String>> {
-
-    private final double samplingThreshold;
-
-    public ComputeTopSessions(double samplingThreshold) {
-      this.samplingThreshold = samplingThreshold;
-    }
-
-    @Override
-    public PCollection<String> apply(PCollection<TableRow> input) {
-      return input
-          .apply(ParDo.of(new ExtractUserAndTimestamp()))
-
-          .apply(ParDo.named("SampleUsers").of(
-              new DoFn<String, String>() {
-                @Override
-                public void processElement(ProcessContext c) {
-                  if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) {
-                    c.output(c.element());
-                  }
-                }
-              }))
-
-          .apply(new ComputeSessions())
-
-          .apply(ParDo.named("SessionsToStrings").of(new SessionsToStringsDoFn()))
-          .apply(new TopPerMonth())
-          .apply(ParDo.named("FormatOutput").of(new FormatOutputDoFn()));
-    }
-  }
-
-  /**
-   * Options supported by this class.
-   *
-   * <p>Inherits standard Dataflow configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description(
-      "Input specified as a GCS path containing a BigQuery table exported as json")
-    @Default.String(EXPORTED_WIKI_TABLE)
-    String getInput();
-    void setInput(String value);
-
-    @Description("File to output results to")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args) {
-    Options options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(Options.class);
-    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-
-    Pipeline p = Pipeline.create(dataflowOptions);
-
-    double samplingThreshold = 0.1;
-
-    p.apply(TextIO.Read
-        .from(options.getInput())
-        .withCoder(TableRowJsonCoder.of()))
-     .apply(new ComputeTopSessions(samplingThreshold))
-     .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput()));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java
deleted file mode 100644
index 2c3c857..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java
+++ /dev/null
@@ -1,426 +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 com.google.cloud.dataflow.examples.complete;
-
-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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Strings;
-
-import org.apache.avro.reflect.Nullable;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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>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>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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
- * and then exits.
- */
-public class TrafficMaxLaneFlow {
-
-  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
-  private static final Integer VALID_INPUTS = 4999;
-
-  static final int WINDOW_DURATION = 60;  // Default sliding window duration in minutes
-  static final int WINDOW_SLIDE_EVERY = 5;  // Default window 'slide every' setting in minutes
-
-  /**
-   * This class holds information about each lane in a station reading, along with some general
-   * information from the reading.
-   */
-  @DefaultCoder(AvroCoder.class)
-  static class LaneInfo {
-    @Nullable String stationId;
-    @Nullable String lane;
-    @Nullable String direction;
-    @Nullable String freeway;
-    @Nullable String recordedTimestamp;
-    @Nullable Integer laneFlow;
-    @Nullable Integer totalFlow;
-    @Nullable Double laneAO;
-    @Nullable Double laneAS;
-
-    public LaneInfo() {}
-
-    public LaneInfo(String stationId, String lane, String direction, String freeway,
-        String timestamp, Integer laneFlow, Double laneAO,
-        Double laneAS, Integer totalFlow) {
-      this.stationId = stationId;
-      this.lane = lane;
-      this.direction = direction;
-      this.freeway = freeway;
-      this.recordedTimestamp = timestamp;
-      this.laneFlow = laneFlow;
-      this.laneAO = laneAO;
-      this.laneAS = laneAS;
-      this.totalFlow = totalFlow;
-    }
-
-    public String getStationId() {
-      return this.stationId;
-    }
-    public String getLane() {
-      return this.lane;
-    }
-    public String getDirection() {
-      return this.direction;
-    }
-    public String getFreeway() {
-      return this.freeway;
-    }
-    public String getRecordedTimestamp() {
-      return this.recordedTimestamp;
-    }
-    public Integer getLaneFlow() {
-      return this.laneFlow;
-    }
-    public Double getLaneAO() {
-      return this.laneAO;
-    }
-    public Double getLaneAS() {
-      return this.laneAS;
-    }
-    public Integer getTotalFlow() {
-      return this.totalFlow;
-    }
-  }
-
-  /**
-   * Extract the timestamp field from the input string, and use it as the element timestamp.
-   */
-  static class ExtractTimestamps extends DoFn<String, String> {
-    private static final DateTimeFormatter dateTimeFormat =
-        DateTimeFormat.forPattern("MM/dd/yyyy HH:mm:ss");
-
-    @Override
-    public void processElement(DoFn<String, String>.ProcessContext c) throws Exception {
-      String[] items = c.element().split(",");
-      if (items.length > 0) {
-        try {
-          String timestamp = items[0];
-          c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp)));
-        } catch (IllegalArgumentException e) {
-          // Skip the invalid input.
-        }
-      }
-    }
-  }
-
-  /**
-   * Extract flow information for each of the 8 lanes in a reading, and output as separate tuples.
-   * This will let us determine which lane has the max flow for that station over the span of the
-   * window, and output not only the max flow from that calculation, but other associated
-   * information. The number of lanes for which data is present depends upon which freeway the data
-   * point comes from.
-   */
-  static class ExtractFlowInfoFn extends DoFn<String, KV<String, LaneInfo>> {
-
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] items = c.element().split(",");
-      if (items.length < 48) {
-        // Skip the invalid input.
-        return;
-      }
-      // extract the sensor information for the lanes from the input string fields.
-      String timestamp = items[0];
-      String stationId = items[1];
-      String freeway = items[2];
-      String direction = items[3];
-      Integer totalFlow = tryIntParse(items[7]);
-      for (int i = 1; i <= 8; ++i) {
-        Integer laneFlow = tryIntParse(items[6 + 5 * i]);
-        Double laneAvgOccupancy = tryDoubleParse(items[7 + 5 * i]);
-        Double laneAvgSpeed = tryDoubleParse(items[8 + 5 * i]);
-        if (laneFlow == null || laneAvgOccupancy == null || laneAvgSpeed == null) {
-          return;
-        }
-        LaneInfo laneInfo = new LaneInfo(stationId, "lane" + i, direction, freeway, timestamp,
-            laneFlow, laneAvgOccupancy, laneAvgSpeed, totalFlow);
-        c.output(KV.of(stationId, laneInfo));
-      }
-    }
-  }
-
-  /**
-   * A custom 'combine function' used with the Combine.perKey transform. Used to find the max lane
-   * flow over all the data points in the Window. Extracts the lane flow from the input string and
-   * determines whether it's the max seen so far. We're using a custom combiner instead of the Max
-   * transform because we want to retain the additional information we've associated with the flow
-   * value.
-   */
-  public static class MaxFlow implements SerializableFunction<Iterable<LaneInfo>, LaneInfo> {
-    @Override
-    public LaneInfo apply(Iterable<LaneInfo> input) {
-      Integer max = 0;
-      LaneInfo maxInfo = new LaneInfo();
-      for (LaneInfo item : input) {
-        Integer flow = item.getLaneFlow();
-        if (flow != null && (flow >= max)) {
-          max = flow;
-          maxInfo = item;
-        }
-      }
-      return maxInfo;
-    }
-  }
-
-  /**
-   * Format the results of the Max Lane flow calculation to a TableRow, to save to BigQuery.
-   * Add the timestamp from the window context.
-   */
-  static class FormatMaxesFn extends DoFn<KV<String, LaneInfo>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-
-      LaneInfo laneInfo = c.element().getValue();
-      TableRow row = new TableRow()
-          .set("station_id", c.element().getKey())
-          .set("direction", laneInfo.getDirection())
-          .set("freeway", laneInfo.getFreeway())
-          .set("lane_max_flow", laneInfo.getLaneFlow())
-          .set("lane", laneInfo.getLane())
-          .set("avg_occ", laneInfo.getLaneAO())
-          .set("avg_speed", laneInfo.getLaneAS())
-          .set("total_flow", laneInfo.getTotalFlow())
-          .set("recorded_timestamp", laneInfo.getRecordedTimestamp())
-          .set("window_timestamp", c.timestamp().toString());
-      c.output(row);
-    }
-
-    /** Defines the BigQuery schema used for the output. */
-    static TableSchema getSchema() {
-      List<TableFieldSchema> fields = new ArrayList<>();
-      fields.add(new TableFieldSchema().setName("station_id").setType("STRING"));
-      fields.add(new TableFieldSchema().setName("direction").setType("STRING"));
-      fields.add(new TableFieldSchema().setName("freeway").setType("STRING"));
-      fields.add(new TableFieldSchema().setName("lane_max_flow").setType("INTEGER"));
-      fields.add(new TableFieldSchema().setName("lane").setType("STRING"));
-      fields.add(new TableFieldSchema().setName("avg_occ").setType("FLOAT"));
-      fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT"));
-      fields.add(new TableFieldSchema().setName("total_flow").setType("INTEGER"));
-      fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
-      fields.add(new TableFieldSchema().setName("recorded_timestamp").setType("STRING"));
-      TableSchema schema = new TableSchema().setFields(fields);
-      return schema;
-    }
-  }
-
-  /**
-   * This PTransform extracts lane info, calculates the max lane flow found for a given station (for
-   * the current Window) using a custom 'combiner', and formats the results for BigQuery.
-   */
-  static class MaxLaneFlow
-      extends PTransform<PCollection<KV<String, LaneInfo>>, PCollection<TableRow>> {
-    @Override
-    public PCollection<TableRow> apply(PCollection<KV<String, LaneInfo>> flowInfo) {
-      // stationId, LaneInfo => stationId + max lane flow info
-      PCollection<KV<String, LaneInfo>> flowMaxes =
-          flowInfo.apply(Combine.<String, LaneInfo>perKey(
-              new MaxFlow()));
-
-      // <stationId, max lane flow info>... => row...
-      PCollection<TableRow> results = flowMaxes.apply(
-          ParDo.of(new FormatMaxesFn()));
-
-      return results;
-    }
-  }
-
-  static class ReadFileAndExtractTimestamps extends PTransform<PBegin, PCollection<String>> {
-    private final String inputFile;
-
-    public ReadFileAndExtractTimestamps(String inputFile) {
-      this.inputFile = inputFile;
-    }
-
-    @Override
-    public PCollection<String> apply(PBegin begin) {
-      return begin
-          .apply(TextIO.Read.from(inputFile))
-          .apply(ParDo.of(new ExtractTimestamps()));
-    }
-  }
-
-  /**
-    * Options supported by {@link TrafficMaxLaneFlow}.
-    *
-    * <p>Inherits standard configuration options.
-    */
-  private interface TrafficMaxLaneFlowOptions extends DataflowExampleOptions,
-      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
-        @Description("Input file to inject to Pub/Sub topic")
-    @Default.String("gs://dataflow-samples/traffic_sensor/"
-        + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
-    String getInputFile();
-    void setInputFile(String value);
-
-    @Description("Numeric value of sliding window duration, in minutes")
-    @Default.Integer(WINDOW_DURATION)
-    Integer getWindowDuration();
-    void setWindowDuration(Integer value);
-
-    @Description("Numeric value of window 'slide every' setting, in minutes")
-    @Default.Integer(WINDOW_SLIDE_EVERY)
-    Integer getWindowSlideEvery();
-    void setWindowSlideEvery(Integer value);
-
-    @Description("Whether to run the pipeline with unbounded input")
-    @Default.Boolean(false)
-    boolean isUnbounded();
-    void setUnbounded(boolean value);
-  }
-
-  /**
-   * Sets up and starts streaming pipeline.
-   *
-   * @throws IOException if there is a problem setting up resources
-   */
-  public static void main(String[] args) throws IOException {
-    TrafficMaxLaneFlowOptions options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(TrafficMaxLaneFlowOptions.class);
-    options.setBigQuerySchema(FormatMaxesFn.getSchema());
-    // Using DataflowExampleUtils to set up required resources.
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options, options.isUnbounded());
-
-    Pipeline pipeline = Pipeline.create(options);
-    TableReference tableRef = new TableReference();
-    tableRef.setProjectId(options.getProject());
-    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
-        // 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.
-        .apply(Window.<KV<String, LaneInfo>>into(SlidingWindows.of(
-            Duration.standardMinutes(options.getWindowDuration())).
-            every(Duration.standardMinutes(options.getWindowSlideEvery()))))
-        .apply(new MaxLaneFlow())
-        .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();
-
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
-    dataflowUtils.waitToFinish(result);
-  }
-
-  private static Integer tryIntParse(String number) {
-    try {
-      return Integer.parseInt(number);
-    } catch (NumberFormatException e) {
-      return null;
-    }
-  }
-
-  private static Double tryDoubleParse(String number) {
-    try {
-      return Double.parseDouble(number);
-    } catch (NumberFormatException e) {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java
deleted file mode 100644
index b1c72e6..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java
+++ /dev/null
@@ -1,460 +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 com.google.cloud.dataflow.examples.complete;
-
-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.cloud.dataflow.examples.common.DataflowExampleOptions;
-import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
-import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
-import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-
-import org.apache.avro.reflect.Nullable;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Hashtable;
-import java.util.List;
-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>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>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 will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
- * and then exits.
- */
-
-public class TrafficRoutes {
-
-  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
-  private static final Integer VALID_INPUTS = 4999;
-
-  // Instantiate some small predefined San Diego routes to analyze
-  static Map<String, String> sdStations = buildStationInfo();
-  static final int WINDOW_DURATION = 3;  // Default sliding window duration in minutes
-  static final int WINDOW_SLIDE_EVERY = 1;  // Default window 'slide every' setting in minutes
-
-  /**
-   * This class holds information about a station reading's average speed.
-   */
-  @DefaultCoder(AvroCoder.class)
-  static class StationSpeed implements Comparable<StationSpeed> {
-    @Nullable String stationId;
-    @Nullable Double avgSpeed;
-    @Nullable Long timestamp;
-
-    public StationSpeed() {}
-
-    public StationSpeed(String stationId, Double avgSpeed, Long timestamp) {
-      this.stationId = stationId;
-      this.avgSpeed = avgSpeed;
-      this.timestamp = timestamp;
-    }
-
-    public String getStationId() {
-      return this.stationId;
-    }
-    public Double getAvgSpeed() {
-      return this.avgSpeed;
-    }
-
-    @Override
-    public int compareTo(StationSpeed other) {
-      return Long.compare(this.timestamp, other.timestamp);
-    }
-  }
-
-  /**
-   * This class holds information about a route's speed/slowdown.
-   */
-  @DefaultCoder(AvroCoder.class)
-  static class RouteInfo {
-    @Nullable String route;
-    @Nullable Double avgSpeed;
-    @Nullable Boolean slowdownEvent;
-
-
-    public RouteInfo() {}
-
-    public RouteInfo(String route, Double avgSpeed, Boolean slowdownEvent) {
-      this.route = route;
-      this.avgSpeed = avgSpeed;
-      this.slowdownEvent = slowdownEvent;
-    }
-
-    public String getRoute() {
-      return this.route;
-    }
-    public Double getAvgSpeed() {
-      return this.avgSpeed;
-    }
-    public Boolean getSlowdownEvent() {
-      return this.slowdownEvent;
-    }
-  }
-
-  /**
-   * Extract the timestamp field from the input string, and use it as the element timestamp.
-   */
-  static class ExtractTimestamps extends DoFn<String, String> {
-    private static final DateTimeFormatter dateTimeFormat =
-        DateTimeFormat.forPattern("MM/dd/yyyy HH:mm:ss");
-
-    @Override
-    public void processElement(DoFn<String, String>.ProcessContext c) throws Exception {
-      String[] items = c.element().split(",");
-      String timestamp = tryParseTimestamp(items);
-      if (timestamp != null) {
-        try {
-          c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp)));
-        } catch (IllegalArgumentException e) {
-          // Skip the invalid input.
-        }
-      }
-    }
-  }
-
-  /**
-   * Filter out readings for the stations along predefined 'routes', and output
-   * (station, speed info) keyed on route.
-   */
-  static class ExtractStationSpeedFn extends DoFn<String, KV<String, StationSpeed>> {
-
-    @Override
-    public void processElement(ProcessContext c) {
-      String[] items = c.element().split(",");
-      String stationType = tryParseStationType(items);
-      // For this analysis, use only 'main line' station types
-      if (stationType != null && stationType.equals("ML")) {
-        Double avgSpeed = tryParseAvgSpeed(items);
-        String stationId = tryParseStationId(items);
-        // For this simple example, filter out everything but some hardwired routes.
-        if (avgSpeed != null && stationId != null && sdStations.containsKey(stationId)) {
-          StationSpeed stationSpeed =
-              new StationSpeed(stationId, avgSpeed, c.timestamp().getMillis());
-          // The tuple key is the 'route' name stored in the 'sdStations' hash.
-          KV<String, StationSpeed> outputValue = KV.of(sdStations.get(stationId), stationSpeed);
-          c.output(outputValue);
-        }
-      }
-    }
-  }
-
-  /**
-   * For a given route, track average speed for the window. Calculate whether
-   * traffic is currently slowing down, via a predefined threshold. If a supermajority of
-   * speeds in this sliding window are less than the previous reading we call this a 'slowdown'.
-   * Note: these calculations are for example purposes only, and are unrealistic and oversimplified.
-   */
-  static class GatherStats
-      extends DoFn<KV<String, Iterable<StationSpeed>>, KV<String, RouteInfo>> {
-    @Override
-    public void processElement(ProcessContext c) throws IOException {
-      String route = c.element().getKey();
-      double speedSum = 0.0;
-      int speedCount = 0;
-      int speedups = 0;
-      int slowdowns = 0;
-      List<StationSpeed> infoList = Lists.newArrayList(c.element().getValue());
-      // StationSpeeds sort by embedded timestamp.
-      Collections.sort(infoList);
-      Map<String, Double> prevSpeeds = new HashMap<>();
-      // For all stations in the route, sum (non-null) speeds. Keep a count of the non-null speeds.
-      for (StationSpeed item : infoList) {
-        Double speed = item.getAvgSpeed();
-        if (speed != null) {
-          speedSum += speed;
-          speedCount++;
-          Double lastSpeed = prevSpeeds.get(item.getStationId());
-          if (lastSpeed != null) {
-            if (lastSpeed < speed) {
-              speedups += 1;
-            } else {
-              slowdowns += 1;
-            }
-          }
-          prevSpeeds.put(item.getStationId(), speed);
-        }
-      }
-      if (speedCount == 0) {
-        // No average to compute.
-        return;
-      }
-      double speedAvg = speedSum / speedCount;
-      boolean slowdownEvent = slowdowns >= 2 * speedups;
-      RouteInfo routeInfo = new RouteInfo(route, speedAvg, slowdownEvent);
-      c.output(KV.of(route, routeInfo));
-    }
-  }
-
-  /**
-   * Format the results of the slowdown calculations to a TableRow, to save to BigQuery.
-   */
-  static class FormatStatsFn extends DoFn<KV<String, RouteInfo>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      RouteInfo routeInfo = c.element().getValue();
-      TableRow row = new TableRow()
-          .set("avg_speed", routeInfo.getAvgSpeed())
-          .set("slowdown_event", routeInfo.getSlowdownEvent())
-          .set("route", c.element().getKey())
-          .set("window_timestamp", c.timestamp().toString());
-      c.output(row);
-    }
-
-    /**
-     * Defines the BigQuery schema used for the output.
-     */
-    static TableSchema getSchema() {
-      List<TableFieldSchema> fields = new ArrayList<>();
-      fields.add(new TableFieldSchema().setName("route").setType("STRING"));
-      fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT"));
-      fields.add(new TableFieldSchema().setName("slowdown_event").setType("BOOLEAN"));
-      fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
-      TableSchema schema = new TableSchema().setFields(fields);
-      return schema;
-    }
-  }
-
-  /**
-   * This PTransform extracts speed info from traffic station readings.
-   * It groups the readings by 'route' and analyzes traffic slowdown for that route.
-   * Lastly, it formats the results for BigQuery.
-   */
-  static class TrackSpeed extends
-      PTransform<PCollection<KV<String, StationSpeed>>, PCollection<TableRow>> {
-    @Override
-    public PCollection<TableRow> apply(PCollection<KV<String, StationSpeed>> stationSpeed) {
-      // Apply a GroupByKey transform to collect a list of all station
-      // readings for a given route.
-      PCollection<KV<String, Iterable<StationSpeed>>> timeGroup = stationSpeed.apply(
-        GroupByKey.<String, StationSpeed>create());
-
-      // Analyze 'slowdown' over the route readings.
-      PCollection<KV<String, RouteInfo>> stats = timeGroup.apply(ParDo.of(new GatherStats()));
-
-      // Format the results for writing to BigQuery
-      PCollection<TableRow> results = stats.apply(
-          ParDo.of(new FormatStatsFn()));
-
-      return results;
-    }
-  }
-
-  static class ReadFileAndExtractTimestamps extends PTransform<PBegin, PCollection<String>> {
-    private final String inputFile;
-
-    public ReadFileAndExtractTimestamps(String inputFile) {
-      this.inputFile = inputFile;
-    }
-
-    @Override
-    public PCollection<String> apply(PBegin begin) {
-      return begin
-          .apply(TextIO.Read.from(inputFile))
-          .apply(ParDo.of(new ExtractTimestamps()));
-    }
-  }
-
-  /**
-  * Options supported by {@link TrafficRoutes}.
-  *
-  * <p>Inherits standard configuration options.
-  */
-  private interface TrafficRoutesOptions extends DataflowExampleOptions,
-      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
-    @Description("Input file to inject to Pub/Sub topic")
-    @Default.String("gs://dataflow-samples/traffic_sensor/"
-        + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
-    String getInputFile();
-    void setInputFile(String value);
-
-    @Description("Numeric value of sliding window duration, in minutes")
-    @Default.Integer(WINDOW_DURATION)
-    Integer getWindowDuration();
-    void setWindowDuration(Integer value);
-
-    @Description("Numeric value of window 'slide every' setting, in minutes")
-    @Default.Integer(WINDOW_SLIDE_EVERY)
-    Integer getWindowSlideEvery();
-    void setWindowSlideEvery(Integer value);
-
-    @Description("Whether to run the pipeline with unbounded input")
-    @Default.Boolean(false)
-    boolean isUnbounded();
-    void setUnbounded(boolean value);
-  }
-
-  /**
-   * Sets up and starts streaming pipeline.
-   *
-   * @throws IOException if there is a problem setting up resources
-   */
-  public static void main(String[] args) throws IOException {
-    TrafficRoutesOptions options = PipelineOptionsFactory.fromArgs(args)
-        .withValidation()
-        .as(TrafficRoutesOptions.class);
-
-    options.setBigQuerySchema(FormatStatsFn.getSchema());
-    // Using DataflowExampleUtils to set up required resources.
-    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options, options.isUnbounded());
-
-    Pipeline pipeline = Pipeline.create(options);
-    TableReference tableRef = new TableReference();
-    tableRef.setProjectId(options.getProject());
-    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
-        // 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()))))
-        .apply(new TrackSpeed())
-        .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();
-
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
-    dataflowUtils.waitToFinish(result);
-  }
-
-  private static Double tryParseAvgSpeed(String[] inputItems) {
-    try {
-      return Double.parseDouble(tryParseString(inputItems, 9));
-    } catch (NumberFormatException e) {
-      return null;
-    } catch (NullPointerException e) {
-      return null;
-    }
-  }
-
-  private static String tryParseStationType(String[] inputItems) {
-    return tryParseString(inputItems, 4);
-  }
-
-  private static String tryParseStationId(String[] inputItems) {
-    return tryParseString(inputItems, 1);
-  }
-
-  private static String tryParseTimestamp(String[] inputItems) {
-    return tryParseString(inputItems, 0);
-  }
-
-  private static String tryParseString(String[] inputItems, int index) {
-    return inputItems.length >= index ? inputItems[index] : null;
-  }
-
-  /**
-   * Define some small hard-wired San Diego 'routes' to track based on sensor station ID.
-   */
-  private static Map<String, String> buildStationInfo() {
-    Map<String, String> stations = new Hashtable<String, String>();
-      stations.put("1108413", "SDRoute1"); // from freeway 805 S
-      stations.put("1108699", "SDRoute2"); // from freeway 78 E
-      stations.put("1108702", "SDRoute2");
-    return stations;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java
deleted file mode 100644
index e5fd015..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java
+++ /dev/null
@@ -1,180 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * An example that reads the public samples of weather data from BigQuery, counts the number of
- * tornadoes that occur in each month, and writes the results to BigQuery.
- *
- * <p>Concepts: Reading/writing BigQuery; counting a PCollection; user-defined PTransforms
- *
- * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
- * table.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output, with the form
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
- * and can be overridden with {@code --input}.
- */
-public class BigQueryTornadoes {
-  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
-  private static final String WEATHER_SAMPLES_TABLE =
-      "clouddataflow-readonly:samples.weather_stations";
-
-  /**
-   * Examines each row in the input table. If a tornado was recorded
-   * in that sample, the month in which it occurred is output.
-   */
-  static class ExtractTornadoesFn extends DoFn<TableRow, Integer> {
-    @Override
-    public void processElement(ProcessContext c){
-      TableRow row = c.element();
-      if ((Boolean) row.get("tornado")) {
-        c.output(Integer.parseInt((String) row.get("month")));
-      }
-    }
-  }
-
-  /**
-   * Prepares the data for writing to BigQuery by building a TableRow object containing an
-   * integer representation of month and the number of tornadoes that occurred in each month.
-   */
-  static class FormatCountsFn extends DoFn<KV<Integer, Long>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = new TableRow()
-          .set("month", c.element().getKey())
-          .set("tornado_count", c.element().getValue());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Takes rows from a table and generates a table of counts.
-   *
-   * <p>The input schema is described by
-   * https://developers.google.com/bigquery/docs/dataset-gsod .
-   * The output contains the total number of tornadoes found in each month in
-   * the following schema:
-   * <ul>
-   *   <li>month: integer</li>
-   *   <li>tornado_count: integer</li>
-   * </ul>
-   */
-  static class CountTornadoes
-      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
-    @Override
-    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
-
-      // row... => month...
-      PCollection<Integer> tornadoes = rows.apply(
-          ParDo.of(new ExtractTornadoesFn()));
-
-      // month... => <month,count>...
-      PCollection<KV<Integer, Long>> tornadoCounts =
-          tornadoes.apply(Count.<Integer>perElement());
-
-      // <month,count>... => row...
-      PCollection<TableRow> results = tornadoCounts.apply(
-          ParDo.of(new FormatCountsFn()));
-
-      return results;
-    }
-  }
-
-  /**
-   * Options supported by {@link BigQueryTornadoes}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Table to read from, specified as "
-        + "<project_id>:<dataset_id>.<table_id>")
-    @Default.String(WEATHER_SAMPLES_TABLE)
-    String getInput();
-    void setInput(String value);
-
-    @Description("BigQuery table to write to, specified as "
-        + "<project_id>:<dataset_id>.<table_id>. The dataset must already exist.")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args) {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
-    Pipeline p = Pipeline.create(options);
-
-    // Build the table schema for the output table.
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("tornado_count").setType("INTEGER"));
-    TableSchema schema = new TableSchema().setFields(fields);
-
-    p.apply(BigQueryIO.Read.from(options.getInput()))
-     .apply(new CountTornadoes())
-     .apply(BigQueryIO.Write
-        .to(options.getOutput())
-        .withSchema(schema)
-        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java
deleted file mode 100644
index 93304eb..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java
+++ /dev/null
@@ -1,224 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * An example that reads the public 'Shakespeare' data, and for each word in
- * the dataset that is over a given length, generates a string containing the
- * list of play names in which that word appears, and saves this information
- * to a bigquery table.
- *
- * <p>Concepts: the Combine.perKey transform, which lets you combine the values in a
- * key-grouped Collection, and how to use an Aggregator to track information in the
- * Monitoring UI.
- *
- * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
- * table.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://<STAGING DIRECTORY>
- *   --runner=BlockingDataflowPipelineRunner
- * }
- * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
- *
- * <p>The BigQuery input table defaults to {@code publicdata:samples.shakespeare} and can
- * be overridden with {@code --input}.
- */
-public class CombinePerKeyExamples {
-  // Use the shakespeare public BigQuery sample
-  private static final String SHAKESPEARE_TABLE =
-      "publicdata:samples.shakespeare";
-  // We'll track words >= this word length across all plays in the table.
-  private static final int MIN_WORD_LENGTH = 9;
-
-  /**
-   * Examines each row in the input table. If the word is greater than or equal to MIN_WORD_LENGTH,
-   * outputs word, play_name.
-   */
-  static class ExtractLargeWordsFn extends DoFn<TableRow, KV<String, String>> {
-    private final Aggregator<Long, Long> smallerWords =
-        createAggregator("smallerWords", new Sum.SumLongFn());
-
-    @Override
-    public void processElement(ProcessContext c){
-      TableRow row = c.element();
-      String playName = (String) row.get("corpus");
-      String word = (String) row.get("word");
-      if (word.length() >= MIN_WORD_LENGTH) {
-        c.output(KV.of(word, playName));
-      } else {
-        // Track how many smaller words we're not including. This information will be
-        // visible in the Monitoring UI.
-        smallerWords.addValue(1L);
-      }
-    }
-  }
-
-
-  /**
-   * Prepares the data for writing to BigQuery by building a TableRow object
-   * containing a word with a string listing the plays in which it appeared.
-   */
-  static class FormatShakespeareOutputFn extends DoFn<KV<String, String>, TableRow> {
-    @Override
-    public void processElement(ProcessContext c) {
-      TableRow row = new TableRow()
-          .set("word", c.element().getKey())
-          .set("all_plays", c.element().getValue());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Reads the public 'Shakespeare' data, and for each word in the dataset
-   * over a given length, generates a string containing the list of play names
-   * in which that word appears. It does this via the Combine.perKey
-   * transform, with the ConcatWords combine function.
-   *
-   * <p>Combine.perKey is similar to a GroupByKey followed by a ParDo, but
-   * has more restricted semantics that allow it to be executed more
-   * efficiently. These records are then formatted as BQ table rows.
-   */
-  static class PlaysForWord
-      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
-    @Override
-    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
-
-      // row... => <word, play_name> ...
-      PCollection<KV<String, String>> words = rows.apply(
-          ParDo.of(new ExtractLargeWordsFn()));
-
-      // word, play_name => word, all_plays ...
-      PCollection<KV<String, String>> wordAllPlays =
-          words.apply(Combine.<String, String>perKey(
-              new ConcatWords()));
-
-      // <word, all_plays>... => row...
-      PCollection<TableRow> results = wordAllPlays.apply(
-          ParDo.of(new FormatShakespeareOutputFn()));
-
-      return results;
-    }
-  }
-
-  /**
-   * A 'combine function' used with the Combine.perKey transform. Builds a
-   * comma-separated string of all input items.  So, it will build a string
-   * containing all the different Shakespeare plays in which the given input
-   * word has appeared.
-   */
-  public static class ConcatWords implements SerializableFunction<Iterable<String>, String> {
-    @Override
-    public String apply(Iterable<String> input) {
-      StringBuilder all = new StringBuilder();
-      for (String item : input) {
-        if (!item.isEmpty()) {
-          if (all.length() == 0) {
-            all.append(item);
-          } else {
-            all.append(",");
-            all.append(item);
-          }
-        }
-      }
-      return all.toString();
-    }
-  }
-
-  /**
-   * Options supported by {@link CombinePerKeyExamples}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Table to read from, specified as "
-        + "<project_id>:<dataset_id>.<table_id>")
-    @Default.String(SHAKESPEARE_TABLE)
-    String getInput();
-    void setInput(String value);
-
-    @Description("Table to write to, specified as "
-        + "<project_id>:<dataset_id>.<table_id>. "
-        + "The dataset_id must already exist")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args)
-      throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-
-    // Build the table schema for the output table.
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("word").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("all_plays").setType("STRING"));
-    TableSchema schema = new TableSchema().setFields(fields);
-
-    p.apply(BigQueryIO.Read.from(options.getInput()))
-     .apply(new PlaysForWord())
-     .apply(BigQueryIO.Write
-        .to(options.getOutput())
-        .withSchema(schema)
-        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java
deleted file mode 100644
index 9dddb5d..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java
+++ /dev/null
@@ -1,270 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import static com.google.api.services.datastore.client.DatastoreHelper.getPropertyMap;
-import static com.google.api.services.datastore.client.DatastoreHelper.getString;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeFilter;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeKey;
-import static com.google.api.services.datastore.client.DatastoreHelper.makeValue;
-
-import com.google.api.services.datastore.DatastoreV1.Entity;
-import com.google.api.services.datastore.DatastoreV1.Key;
-import com.google.api.services.datastore.DatastoreV1.Property;
-import com.google.api.services.datastore.DatastoreV1.PropertyFilter;
-import com.google.api.services.datastore.DatastoreV1.Query;
-import com.google.api.services.datastore.DatastoreV1.Value;
-import com.google.cloud.dataflow.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.DatastoreIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-
-import java.util.Map;
-import java.util.UUID;
-
-import javax.annotation.Nullable;
-
-/**
- * A WordCount example using DatastoreIO.
- *
- * <p>This example shows how to use DatastoreIO to read from Datastore and
- * write the results to Cloud Storage.  Note that this example will write
- * data to Datastore, which may incur charge for Datastore operations.
- *
- * <p>To run this example, users need to use gcloud to get credential for Datastore:
- * <pre>{@code
- * $ gcloud auth login
- * }</pre>
- *
- * <p>To run this pipeline locally, the following options must be provided:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --dataset=YOUR_DATASET_ID
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PATH]
- * }</pre>
- *
- * <p>To run this example using Dataflow service, you must additionally
- * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and
- * select one of the Dataflow pipeline runners, eg
- * {@literal --runner=BlockingDataflowPipelineRunner}.
- *
- * <p><b>Note:</b> this example creates entities with <i>Ancestor keys</i> to ensure that all
- * entities created are in the same entity group. Similarly, the query used to read from the Cloud
- * Datastore uses an <i>Ancestor filter</i>. Ancestors are used to ensure strongly consistent
- * results in Cloud Datastore. For more information, see the Cloud Datastore documentation on
- * <a href="https://cloud.google.com/datastore/docs/concepts/structuring_for_strong_consistency">
- * Structing Data for Strong Consistency</a>.
- */
-public class DatastoreWordCount {
-
-  /**
-   * A DoFn that gets the content of an entity (one line in a
-   * Shakespeare play) and converts it to a string.
-   */
-  static class GetContentFn extends DoFn<Entity, String> {
-    @Override
-    public void processElement(ProcessContext c) {
-      Map<String, Value> props = getPropertyMap(c.element());
-      Value value = props.get("content");
-      if (value != null) {
-        c.output(getString(value));
-      }
-    }
-  }
-
-  /**
-   * A helper function to create the ancestor key for all created and queried entities.
-   *
-   * <p>We use ancestor keys and ancestor queries for strong consistency. See
-   * {@link DatastoreWordCount} javadoc for more information.
-   */
-  static Key makeAncestorKey(@Nullable String namespace, String kind) {
-    Key.Builder keyBuilder = makeKey(kind, "root");
-    if (namespace != null) {
-      keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
-    }
-    return keyBuilder.build();
-  }
-
-  /**
-   * A DoFn that creates entity for every line in Shakespeare.
-   */
-  static class CreateEntityFn extends DoFn<String, Entity> {
-    private final String namespace;
-    private final String kind;
-    private final Key ancestorKey;
-
-    CreateEntityFn(String namespace, String kind) {
-      this.namespace = namespace;
-      this.kind = kind;
-
-      // Build the ancestor key for all created entities once, including the namespace.
-      ancestorKey = makeAncestorKey(namespace, kind);
-    }
-
-    public Entity makeEntity(String content) {
-      Entity.Builder entityBuilder = Entity.newBuilder();
-
-      // All created entities have the same ancestor Key.
-      Key.Builder keyBuilder = makeKey(ancestorKey, kind, UUID.randomUUID().toString());
-      // NOTE: Namespace is not inherited between keys created with DatastoreHelper.makeKey, so
-      // we must set the namespace on keyBuilder. TODO: Once partitionId inheritance is added,
-      // we can simplify this code.
-      if (namespace != null) {
-        keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
-      }
-
-      entityBuilder.setKey(keyBuilder.build());
-      entityBuilder.addProperty(Property.newBuilder().setName("content")
-          .setValue(Value.newBuilder().setStringValue(content)));
-      return entityBuilder.build();
-    }
-
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(makeEntity(c.element()));
-    }
-  }
-
-  /**
-   * Options supported by {@link DatastoreWordCount}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  public static interface Options extends PipelineOptions {
-    @Description("Path of the file to read from and store to Datastore")
-    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Path of the file to write to")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-
-    @Description("Dataset ID to read from datastore")
-    @Validation.Required
-    String getDataset();
-    void setDataset(String value);
-
-    @Description("Dataset entity kind")
-    @Default.String("shakespeare-demo")
-    String getKind();
-    void setKind(String value);
-
-    @Description("Dataset namespace")
-    String getNamespace();
-    void setNamespace(@Nullable String value);
-
-    @Description("Read an existing dataset, do not write first")
-    boolean isReadOnly();
-    void setReadOnly(boolean value);
-
-    @Description("Number of output shards")
-    @Default.Integer(0) // If the system should choose automatically.
-    int getNumShards();
-    void setNumShards(int value);
-  }
-
-  /**
-   * An example that creates a pipeline to populate DatastoreIO from a
-   * text input.  Forces use of DirectPipelineRunner for local execution mode.
-   */
-  public static void writeDataToDatastore(Options options) {
-      Pipeline p = Pipeline.create(options);
-      p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
-       .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind())))
-       .apply(DatastoreIO.writeTo(options.getDataset()));
-
-      p.run();
-  }
-
-  /**
-   * Build a Cloud Datastore ancestor query for the specified {@link Options#getNamespace} and
-   * {@link Options#getKind}.
-   *
-   * <p>We use ancestor keys and ancestor queries for strong consistency. See
-   * {@link DatastoreWordCount} javadoc for more information.
-   *
-   * @see <a href="https://cloud.google.com/datastore/docs/concepts/queries#Datastore_Ancestor_filters">Ancestor filters</a>
-   */
-  static Query makeAncestorKindQuery(Options options) {
-    Query.Builder q = Query.newBuilder();
-    q.addKindBuilder().setName(options.getKind());
-    q.setFilter(makeFilter(
-        "__key__",
-        PropertyFilter.Operator.HAS_ANCESTOR,
-        makeValue(makeAncestorKey(options.getNamespace(), options.getKind()))));
-    return q.build();
-  }
-
-  /**
-   * An example that creates a pipeline to do DatastoreIO.Read from Datastore.
-   */
-  public static void readDataFromDatastore(Options options) {
-    Query query = makeAncestorKindQuery(options);
-
-    // For Datastore sources, the read namespace can be set on the entire query.
-    DatastoreIO.Source source = DatastoreIO.source()
-        .withDataset(options.getDataset())
-        .withQuery(query)
-        .withNamespace(options.getNamespace());
-
-    Pipeline p = Pipeline.create(options);
-    p.apply("ReadShakespeareFromDatastore", Read.from(source))
-        .apply("StringifyEntity", ParDo.of(new GetContentFn()))
-        .apply("CountWords", new WordCount.CountWords())
-        .apply("PrintWordCount", MapElements.via(new WordCount.FormatAsTextFn()))
-        .apply("WriteLines", TextIO.Write.to(options.getOutput())
-            .withNumShards(options.getNumShards()));
-    p.run();
-  }
-
-  /**
-   * An example to demo how to use {@link DatastoreIO}.  The runner here is
-   * customizable, which means users could pass either {@code DirectPipelineRunner}
-   * or {@code DataflowPipelineRunner} in the pipeline options.
-   */
-  public static void main(String args[]) {
-    // The options are used in two places, for Dataflow service, and
-    // building DatastoreIO.Read object
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
-    if (!options.isReadOnly()) {
-      // First example: write data to Datastore for reading later.
-      //
-      // NOTE: this write does not delete any existing Entities in the Datastore, so if run
-      // multiple times with the same output dataset, there may be duplicate entries. The
-      // Datastore Query tool in the Google Developers Console can be used to inspect or erase all
-      // entries with a particular namespace and/or kind.
-      DatastoreWordCount.writeDataToDatastore(options);
-    }
-
-    // Second example: do parallel read from Datastore.
-    DatastoreWordCount.readDataFromDatastore(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java
deleted file mode 100644
index 40d1f76..0000000
--- a/examples/java/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java
+++ /dev/null
@@ -1,101 +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 com.google.cloud.dataflow.examples.cookbook;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-
-/**
- * This example uses as input Shakespeare's plays as plaintext files, and will remove any
- * duplicate lines across all the files. (The output does not preserve any input order).
- *
- * <p>Concepts: the RemoveDuplicates transform, and how to wire transforms together.
- * Demonstrates {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}/
- * {@link RemoveDuplicates}/{@link com.google.cloud.dataflow.sdk.io.TextIO.Write}.
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- *   --project=YOUR_PROJECT_ID
- * and a local output file or output prefix on GCS:
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
- * and an output prefix on GCS:
- *   --output=gs://YOUR_OUTPUT_PREFIX
- *
- * <p>The input defaults to {@code gs://dataflow-samples/shakespeare/*} and can be
- * overridden with {@code --input}.
- */
-public class DeDupExample {
-
-  /**
-   * Options supported by {@link DeDupExample}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private static interface Options extends PipelineOptions {
-    @Description("Path to the directory or GCS prefix containing files to read from")
-    @Default.String("gs://dataflow-samples/shakespeare/*")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Path of the file to write to")
-    @Default.InstanceFactory(OutputFactory.class)
-    String getOutput();
-    void setOutput(String value);
-
-    /** Returns gs://${STAGING_LOCATION}/"deduped.txt". */
-    public static class OutputFactory implements DefaultValueFactory<String> {
-      @Override
-      public String create(PipelineOptions options) {
-        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-        if (dataflowOptions.getStagingLocation() != null) {
-          return GcsPath.fromUri(dataflowOptions.getStagingLocation())
-              .resolve("deduped.txt").toString();
-        } else {
-          throw new IllegalArgumentException("Must specify --output or --stagingLocation");
-        }
-      }
-    }
-  }
-
-
-  public static void main(String[] args)
-      throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-
-    p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
-     .apply(RemoveDuplicates.<String>create())
-     .apply(TextIO.Write.named("DedupedShakespeare")
-         .to(options.getOutput()));
-
-    p.run();
-  }
-}


[43/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
new file mode 100644
index 0000000..93304eb
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
@@ -0,0 +1,224 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An example that reads the public 'Shakespeare' data, and for each word in
+ * the dataset that is over a given length, generates a string containing the
+ * list of play names in which that word appears, and saves this information
+ * to a bigquery table.
+ *
+ * <p>Concepts: the Combine.perKey transform, which lets you combine the values in a
+ * key-grouped Collection, and how to use an Aggregator to track information in the
+ * Monitoring UI.
+ *
+ * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
+ * table.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://<STAGING DIRECTORY>
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>The BigQuery input table defaults to {@code publicdata:samples.shakespeare} and can
+ * be overridden with {@code --input}.
+ */
+public class CombinePerKeyExamples {
+  // Use the shakespeare public BigQuery sample
+  private static final String SHAKESPEARE_TABLE =
+      "publicdata:samples.shakespeare";
+  // We'll track words >= this word length across all plays in the table.
+  private static final int MIN_WORD_LENGTH = 9;
+
+  /**
+   * Examines each row in the input table. If the word is greater than or equal to MIN_WORD_LENGTH,
+   * outputs word, play_name.
+   */
+  static class ExtractLargeWordsFn extends DoFn<TableRow, KV<String, String>> {
+    private final Aggregator<Long, Long> smallerWords =
+        createAggregator("smallerWords", new Sum.SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c){
+      TableRow row = c.element();
+      String playName = (String) row.get("corpus");
+      String word = (String) row.get("word");
+      if (word.length() >= MIN_WORD_LENGTH) {
+        c.output(KV.of(word, playName));
+      } else {
+        // Track how many smaller words we're not including. This information will be
+        // visible in the Monitoring UI.
+        smallerWords.addValue(1L);
+      }
+    }
+  }
+
+
+  /**
+   * Prepares the data for writing to BigQuery by building a TableRow object
+   * containing a word with a string listing the plays in which it appeared.
+   */
+  static class FormatShakespeareOutputFn extends DoFn<KV<String, String>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = new TableRow()
+          .set("word", c.element().getKey())
+          .set("all_plays", c.element().getValue());
+      c.output(row);
+    }
+  }
+
+  /**
+   * Reads the public 'Shakespeare' data, and for each word in the dataset
+   * over a given length, generates a string containing the list of play names
+   * in which that word appears. It does this via the Combine.perKey
+   * transform, with the ConcatWords combine function.
+   *
+   * <p>Combine.perKey is similar to a GroupByKey followed by a ParDo, but
+   * has more restricted semantics that allow it to be executed more
+   * efficiently. These records are then formatted as BQ table rows.
+   */
+  static class PlaysForWord
+      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
+    @Override
+    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
+
+      // row... => <word, play_name> ...
+      PCollection<KV<String, String>> words = rows.apply(
+          ParDo.of(new ExtractLargeWordsFn()));
+
+      // word, play_name => word, all_plays ...
+      PCollection<KV<String, String>> wordAllPlays =
+          words.apply(Combine.<String, String>perKey(
+              new ConcatWords()));
+
+      // <word, all_plays>... => row...
+      PCollection<TableRow> results = wordAllPlays.apply(
+          ParDo.of(new FormatShakespeareOutputFn()));
+
+      return results;
+    }
+  }
+
+  /**
+   * A 'combine function' used with the Combine.perKey transform. Builds a
+   * comma-separated string of all input items.  So, it will build a string
+   * containing all the different Shakespeare plays in which the given input
+   * word has appeared.
+   */
+  public static class ConcatWords implements SerializableFunction<Iterable<String>, String> {
+    @Override
+    public String apply(Iterable<String> input) {
+      StringBuilder all = new StringBuilder();
+      for (String item : input) {
+        if (!item.isEmpty()) {
+          if (all.length() == 0) {
+            all.append(item);
+          } else {
+            all.append(",");
+            all.append(item);
+          }
+        }
+      }
+      return all.toString();
+    }
+  }
+
+  /**
+   * Options supported by {@link CombinePerKeyExamples}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Table to read from, specified as "
+        + "<project_id>:<dataset_id>.<table_id>")
+    @Default.String(SHAKESPEARE_TABLE)
+    String getInput();
+    void setInput(String value);
+
+    @Description("Table to write to, specified as "
+        + "<project_id>:<dataset_id>.<table_id>. "
+        + "The dataset_id must already exist")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  public static void main(String[] args)
+      throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+
+    // Build the table schema for the output table.
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("word").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("all_plays").setType("STRING"));
+    TableSchema schema = new TableSchema().setFields(fields);
+
+    p.apply(BigQueryIO.Read.from(options.getInput()))
+     .apply(new PlaysForWord())
+     .apply(BigQueryIO.Write
+        .to(options.getOutput())
+        .withSchema(schema)
+        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..9dddb5d
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
@@ -0,0 +1,270 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import static com.google.api.services.datastore.client.DatastoreHelper.getPropertyMap;
+import static com.google.api.services.datastore.client.DatastoreHelper.getString;
+import static com.google.api.services.datastore.client.DatastoreHelper.makeFilter;
+import static com.google.api.services.datastore.client.DatastoreHelper.makeKey;
+import static com.google.api.services.datastore.client.DatastoreHelper.makeValue;
+
+import com.google.api.services.datastore.DatastoreV1.Entity;
+import com.google.api.services.datastore.DatastoreV1.Key;
+import com.google.api.services.datastore.DatastoreV1.Property;
+import com.google.api.services.datastore.DatastoreV1.PropertyFilter;
+import com.google.api.services.datastore.DatastoreV1.Query;
+import com.google.api.services.datastore.DatastoreV1.Value;
+import com.google.cloud.dataflow.examples.WordCount;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.DatastoreIO;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+
+import java.util.Map;
+import java.util.UUID;
+
+import javax.annotation.Nullable;
+
+/**
+ * A WordCount example using DatastoreIO.
+ *
+ * <p>This example shows how to use DatastoreIO to read from Datastore and
+ * write the results to Cloud Storage.  Note that this example will write
+ * data to Datastore, which may incur charge for Datastore operations.
+ *
+ * <p>To run this example, users need to use gcloud to get credential for Datastore:
+ * <pre>{@code
+ * $ gcloud auth login
+ * }</pre>
+ *
+ * <p>To run this pipeline locally, the following options must be provided:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --dataset=YOUR_DATASET_ID
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PATH]
+ * }</pre>
+ *
+ * <p>To run this example using Dataflow service, you must additionally
+ * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and
+ * select one of the Dataflow pipeline runners, eg
+ * {@literal --runner=BlockingDataflowPipelineRunner}.
+ *
+ * <p><b>Note:</b> this example creates entities with <i>Ancestor keys</i> to ensure that all
+ * entities created are in the same entity group. Similarly, the query used to read from the Cloud
+ * Datastore uses an <i>Ancestor filter</i>. Ancestors are used to ensure strongly consistent
+ * results in Cloud Datastore. For more information, see the Cloud Datastore documentation on
+ * <a href="https://cloud.google.com/datastore/docs/concepts/structuring_for_strong_consistency">
+ * Structing Data for Strong Consistency</a>.
+ */
+public class DatastoreWordCount {
+
+  /**
+   * A DoFn that gets the content of an entity (one line in a
+   * Shakespeare play) and converts it to a string.
+   */
+  static class GetContentFn extends DoFn<Entity, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      Map<String, Value> props = getPropertyMap(c.element());
+      Value value = props.get("content");
+      if (value != null) {
+        c.output(getString(value));
+      }
+    }
+  }
+
+  /**
+   * A helper function to create the ancestor key for all created and queried entities.
+   *
+   * <p>We use ancestor keys and ancestor queries for strong consistency. See
+   * {@link DatastoreWordCount} javadoc for more information.
+   */
+  static Key makeAncestorKey(@Nullable String namespace, String kind) {
+    Key.Builder keyBuilder = makeKey(kind, "root");
+    if (namespace != null) {
+      keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
+    }
+    return keyBuilder.build();
+  }
+
+  /**
+   * A DoFn that creates entity for every line in Shakespeare.
+   */
+  static class CreateEntityFn extends DoFn<String, Entity> {
+    private final String namespace;
+    private final String kind;
+    private final Key ancestorKey;
+
+    CreateEntityFn(String namespace, String kind) {
+      this.namespace = namespace;
+      this.kind = kind;
+
+      // Build the ancestor key for all created entities once, including the namespace.
+      ancestorKey = makeAncestorKey(namespace, kind);
+    }
+
+    public Entity makeEntity(String content) {
+      Entity.Builder entityBuilder = Entity.newBuilder();
+
+      // All created entities have the same ancestor Key.
+      Key.Builder keyBuilder = makeKey(ancestorKey, kind, UUID.randomUUID().toString());
+      // NOTE: Namespace is not inherited between keys created with DatastoreHelper.makeKey, so
+      // we must set the namespace on keyBuilder. TODO: Once partitionId inheritance is added,
+      // we can simplify this code.
+      if (namespace != null) {
+        keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
+      }
+
+      entityBuilder.setKey(keyBuilder.build());
+      entityBuilder.addProperty(Property.newBuilder().setName("content")
+          .setValue(Value.newBuilder().setStringValue(content)));
+      return entityBuilder.build();
+    }
+
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(makeEntity(c.element()));
+    }
+  }
+
+  /**
+   * Options supported by {@link DatastoreWordCount}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  public static interface Options extends PipelineOptions {
+    @Description("Path of the file to read from and store to Datastore")
+    @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
+    String getInput();
+    void setInput(String value);
+
+    @Description("Path of the file to write to")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+
+    @Description("Dataset ID to read from datastore")
+    @Validation.Required
+    String getDataset();
+    void setDataset(String value);
+
+    @Description("Dataset entity kind")
+    @Default.String("shakespeare-demo")
+    String getKind();
+    void setKind(String value);
+
+    @Description("Dataset namespace")
+    String getNamespace();
+    void setNamespace(@Nullable String value);
+
+    @Description("Read an existing dataset, do not write first")
+    boolean isReadOnly();
+    void setReadOnly(boolean value);
+
+    @Description("Number of output shards")
+    @Default.Integer(0) // If the system should choose automatically.
+    int getNumShards();
+    void setNumShards(int value);
+  }
+
+  /**
+   * An example that creates a pipeline to populate DatastoreIO from a
+   * text input.  Forces use of DirectPipelineRunner for local execution mode.
+   */
+  public static void writeDataToDatastore(Options options) {
+      Pipeline p = Pipeline.create(options);
+      p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+       .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind())))
+       .apply(DatastoreIO.writeTo(options.getDataset()));
+
+      p.run();
+  }
+
+  /**
+   * Build a Cloud Datastore ancestor query for the specified {@link Options#getNamespace} and
+   * {@link Options#getKind}.
+   *
+   * <p>We use ancestor keys and ancestor queries for strong consistency. See
+   * {@link DatastoreWordCount} javadoc for more information.
+   *
+   * @see <a href="https://cloud.google.com/datastore/docs/concepts/queries#Datastore_Ancestor_filters">Ancestor filters</a>
+   */
+  static Query makeAncestorKindQuery(Options options) {
+    Query.Builder q = Query.newBuilder();
+    q.addKindBuilder().setName(options.getKind());
+    q.setFilter(makeFilter(
+        "__key__",
+        PropertyFilter.Operator.HAS_ANCESTOR,
+        makeValue(makeAncestorKey(options.getNamespace(), options.getKind()))));
+    return q.build();
+  }
+
+  /**
+   * An example that creates a pipeline to do DatastoreIO.Read from Datastore.
+   */
+  public static void readDataFromDatastore(Options options) {
+    Query query = makeAncestorKindQuery(options);
+
+    // For Datastore sources, the read namespace can be set on the entire query.
+    DatastoreIO.Source source = DatastoreIO.source()
+        .withDataset(options.getDataset())
+        .withQuery(query)
+        .withNamespace(options.getNamespace());
+
+    Pipeline p = Pipeline.create(options);
+    p.apply("ReadShakespeareFromDatastore", Read.from(source))
+        .apply("StringifyEntity", ParDo.of(new GetContentFn()))
+        .apply("CountWords", new WordCount.CountWords())
+        .apply("PrintWordCount", MapElements.via(new WordCount.FormatAsTextFn()))
+        .apply("WriteLines", TextIO.Write.to(options.getOutput())
+            .withNumShards(options.getNumShards()));
+    p.run();
+  }
+
+  /**
+   * An example to demo how to use {@link DatastoreIO}.  The runner here is
+   * customizable, which means users could pass either {@code DirectPipelineRunner}
+   * or {@code DataflowPipelineRunner} in the pipeline options.
+   */
+  public static void main(String args[]) {
+    // The options are used in two places, for Dataflow service, and
+    // building DatastoreIO.Read object
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+    if (!options.isReadOnly()) {
+      // First example: write data to Datastore for reading later.
+      //
+      // NOTE: this write does not delete any existing Entities in the Datastore, so if run
+      // multiple times with the same output dataset, there may be duplicate entries. The
+      // Datastore Query tool in the Google Developers Console can be used to inspect or erase all
+      // entries with a particular namespace and/or kind.
+      DatastoreWordCount.writeDataToDatastore(options);
+    }
+
+    // Second example: do parallel read from Datastore.
+    DatastoreWordCount.readDataFromDatastore(options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..40d1f76
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
@@ -0,0 +1,101 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+
+/**
+ * This example uses as input Shakespeare's plays as plaintext files, and will remove any
+ * duplicate lines across all the files. (The output does not preserve any input order).
+ *
+ * <p>Concepts: the RemoveDuplicates transform, and how to wire transforms together.
+ * Demonstrates {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}/
+ * {@link RemoveDuplicates}/{@link com.google.cloud.dataflow.sdk.io.TextIO.Write}.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ *   --project=YOUR_PROJECT_ID
+ * and a local output file or output prefix on GCS:
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * and an output prefix on GCS:
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ *
+ * <p>The input defaults to {@code gs://dataflow-samples/shakespeare/*} and can be
+ * overridden with {@code --input}.
+ */
+public class DeDupExample {
+
+  /**
+   * Options supported by {@link DeDupExample}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Path to the directory or GCS prefix containing files to read from")
+    @Default.String("gs://dataflow-samples/shakespeare/*")
+    String getInput();
+    void setInput(String value);
+
+    @Description("Path of the file to write to")
+    @Default.InstanceFactory(OutputFactory.class)
+    String getOutput();
+    void setOutput(String value);
+
+    /** Returns gs://${STAGING_LOCATION}/"deduped.txt". */
+    public static class OutputFactory implements DefaultValueFactory<String> {
+      @Override
+      public String create(PipelineOptions options) {
+        DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+        if (dataflowOptions.getStagingLocation() != null) {
+          return GcsPath.fromUri(dataflowOptions.getStagingLocation())
+              .resolve("deduped.txt").toString();
+        } else {
+          throw new IllegalArgumentException("Must specify --output or --stagingLocation");
+        }
+      }
+    }
+  }
+
+
+  public static void main(String[] args)
+      throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+
+    p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+     .apply(RemoveDuplicates.<String>create())
+     .apply(TextIO.Write.named("DedupedShakespeare")
+         .to(options.getOutput()));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
new file mode 100644
index 0000000..269ee6a
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
@@ -0,0 +1,267 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Mean;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+/**
+ * This is an example that demonstrates several approaches to filtering, and use of the Mean
+ * transform. It shows how to dynamically set parameters by defining and using new pipeline options,
+ * and how to use a value derived by the pipeline.
+ *
+ * <p>Concepts: The Mean transform; Options configuration; using pipeline-derived data as a side
+ * input; approaches to filtering, selection, and projection.
+ *
+ * <p>The example reads public samples of weather data from BigQuery. It performs a
+ * projection on the data, finds the global mean of the temperature readings, filters on readings
+ * for a single given month, and then outputs only data (for that month) that has a mean temp
+ * smaller than the derived global mean.
+*
+ * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
+ * table.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ *   [--monthFilter=<month_number>]
+ * }
+ * </pre>
+ * where optional parameter {@code --monthFilter} is set to a number 1-12.
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ *   [--monthFilter=<month_number>]
+ * }
+ * </pre>
+ * where optional parameter {@code --monthFilter} is set to a number 1-12.
+ *
+ * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
+ * and can be overridden with {@code --input}.
+ */
+public class FilterExamples {
+  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
+  private static final String WEATHER_SAMPLES_TABLE =
+      "clouddataflow-readonly:samples.weather_stations";
+  static final Logger LOG = Logger.getLogger(FilterExamples.class.getName());
+  static final int MONTH_TO_FILTER = 7;
+
+  /**
+   * Examines each row in the input table. Outputs only the subset of the cells this example
+   * is interested in-- the mean_temp and year, month, and day-- as a bigquery table row.
+   */
+  static class ProjectionFn extends DoFn<TableRow, TableRow> {
+    @Override
+    public void processElement(ProcessContext c){
+      TableRow row = c.element();
+      // Grab year, month, day, mean_temp from the row
+      Integer year = Integer.parseInt((String) row.get("year"));
+      Integer month = Integer.parseInt((String) row.get("month"));
+      Integer day = Integer.parseInt((String) row.get("day"));
+      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
+      // Prepares the data for writing to BigQuery by building a TableRow object
+      TableRow outRow = new TableRow()
+          .set("year", year).set("month", month)
+          .set("day", day).set("mean_temp", meanTemp);
+      c.output(outRow);
+    }
+  }
+
+  /**
+   * Implements 'filter' functionality.
+   *
+   * <p>Examines each row in the input table. Outputs only rows from the month
+   * monthFilter, which is passed in as a parameter during construction of this DoFn.
+   */
+  static class FilterSingleMonthDataFn extends DoFn<TableRow, TableRow> {
+    Integer monthFilter;
+
+    public FilterSingleMonthDataFn(Integer monthFilter) {
+      this.monthFilter = monthFilter;
+    }
+
+    @Override
+    public void processElement(ProcessContext c){
+      TableRow row = c.element();
+      Integer month;
+      month = (Integer) row.get("month");
+      if (month.equals(this.monthFilter)) {
+        c.output(row);
+      }
+    }
+  }
+
+  /**
+   * Examines each row (weather reading) in the input table. Output the temperature
+   * reading for that row ('mean_temp').
+   */
+  static class ExtractTempFn extends DoFn<TableRow, Double> {
+    @Override
+    public void processElement(ProcessContext c){
+      TableRow row = c.element();
+      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
+      c.output(meanTemp);
+    }
+  }
+
+
+
+  /*
+   * Finds the global mean of the mean_temp for each day/record, and outputs
+   * only data that has a mean temp larger than this global mean.
+   **/
+  static class BelowGlobalMean
+      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
+    Integer monthFilter;
+
+    public BelowGlobalMean(Integer monthFilter) {
+      this.monthFilter = monthFilter;
+    }
+
+
+    @Override
+    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
+
+      // Extract the mean_temp from each row.
+      PCollection<Double> meanTemps = rows.apply(
+          ParDo.of(new ExtractTempFn()));
+
+      // Find the global mean, of all the mean_temp readings in the weather data,
+      // and prepare this singleton PCollectionView for use as a side input.
+      final PCollectionView<Double> globalMeanTemp =
+          meanTemps.apply(Mean.<Double>globally())
+               .apply(View.<Double>asSingleton());
+
+      // Rows filtered to remove all but a single month
+      PCollection<TableRow> monthFilteredRows = rows
+          .apply(ParDo.of(new FilterSingleMonthDataFn(monthFilter)));
+
+      // Then, use the global mean as a side input, to further filter the weather data.
+      // By using a side input to pass in the filtering criteria, we can use a value
+      // that is computed earlier in pipeline execution.
+      // We'll only output readings with temperatures below this mean.
+      PCollection<TableRow> filteredRows = monthFilteredRows
+          .apply(ParDo
+              .named("ParseAndFilter")
+              .withSideInputs(globalMeanTemp)
+              .of(new DoFn<TableRow, TableRow>() {
+                @Override
+                public void processElement(ProcessContext c) {
+                  Double meanTemp = Double.parseDouble(c.element().get("mean_temp").toString());
+                  Double gTemp = c.sideInput(globalMeanTemp);
+                  if (meanTemp < gTemp) {
+                    c.output(c.element());
+                  }
+                }
+              }));
+
+      return filteredRows;
+    }
+  }
+
+
+  /**
+   * Options supported by {@link FilterExamples}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Table to read from, specified as "
+        + "<project_id>:<dataset_id>.<table_id>")
+    @Default.String(WEATHER_SAMPLES_TABLE)
+    String getInput();
+    void setInput(String value);
+
+    @Description("Table to write to, specified as "
+        + "<project_id>:<dataset_id>.<table_id>. "
+        + "The dataset_id must already exist")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+
+    @Description("Numeric value of month to filter on")
+    @Default.Integer(MONTH_TO_FILTER)
+    Integer getMonthFilter();
+    void setMonthFilter(Integer value);
+  }
+
+  /**
+   * Helper method to build the table schema for the output table.
+   */
+  private static TableSchema buildWeatherSchemaProjection() {
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("year").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("day").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("mean_temp").setType("FLOAT"));
+    TableSchema schema = new TableSchema().setFields(fields);
+    return schema;
+  }
+
+  public static void main(String[] args)
+      throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+
+    TableSchema schema = buildWeatherSchemaProjection();
+
+    p.apply(BigQueryIO.Read.from(options.getInput()))
+     .apply(ParDo.of(new ProjectionFn()))
+     .apply(new BelowGlobalMean(options.getMonthFilter()))
+     .apply(BigQueryIO.Write
+        .to(options.getOutput())
+        .withSchema(schema)
+        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
new file mode 100644
index 0000000..765420e
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
@@ -0,0 +1,186 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+
+/**
+ * This example shows how to do a join on two collections.
+ * It uses a sample of the GDELT 'world event' data (http://goo.gl/OB6oin), joining the event
+ * 'action' country code against a table that maps country codes to country names.
+ *
+ * <p>Concepts: Join operation; multiple input sources.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and a local output file or output prefix on GCS:
+ * <pre>{@code
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and an output prefix on GCS:
+ * <pre>{@code
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ * }</pre>
+ */
+public class JoinExamples {
+
+  // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
+  private static final String GDELT_EVENTS_TABLE =
+      "clouddataflow-readonly:samples.gdelt_sample";
+  // A table that maps country codes to country names.
+  private static final String COUNTRY_CODES =
+      "gdelt-bq:full.crosswalk_geocountrycodetohuman";
+
+  /**
+   * Join two collections, using country code as the key.
+   */
+  static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
+      PCollection<TableRow> countryCodes) throws Exception {
+
+    final TupleTag<String> eventInfoTag = new TupleTag<String>();
+    final TupleTag<String> countryInfoTag = new TupleTag<String>();
+
+    // transform both input collections to tuple collections, where the keys are country
+    // codes in both cases.
+    PCollection<KV<String, String>> eventInfo = eventsTable.apply(
+        ParDo.of(new ExtractEventDataFn()));
+    PCollection<KV<String, String>> countryInfo = countryCodes.apply(
+        ParDo.of(new ExtractCountryInfoFn()));
+
+    // country code 'key' -> CGBKR (<event info>, <country name>)
+    PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+        .of(eventInfoTag, eventInfo)
+        .and(countryInfoTag, countryInfo)
+        .apply(CoGroupByKey.<String>create());
+
+    // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+    // country code 'key' -> string of <event info>, <country name>
+    PCollection<KV<String, String>> finalResultCollection =
+      kvpCollection.apply(ParDo.named("Process").of(
+        new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+          @Override
+          public void processElement(ProcessContext c) {
+            KV<String, CoGbkResult> e = c.element();
+            String countryCode = e.getKey();
+            String countryName = "none";
+            countryName = e.getValue().getOnly(countryInfoTag);
+            for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
+              // Generate a string that combines information from both collection values
+              c.output(KV.of(countryCode, "Country name: " + countryName
+                      + ", Event info: " + eventInfo));
+            }
+          }
+      }));
+
+    // write to GCS
+    PCollection<String> formattedResults = finalResultCollection
+        .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
+          @Override
+          public void processElement(ProcessContext c) {
+            String outputstring = "Country code: " + c.element().getKey()
+                + ", " + c.element().getValue();
+            c.output(outputstring);
+          }
+        }));
+    return formattedResults;
+  }
+
+  /**
+   * Examines each row (event) in the input table. Output a KV with the key the country
+   * code of the event, and the value a string encoding event information.
+   */
+  static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = c.element();
+      String countryCode = (String) row.get("ActionGeo_CountryCode");
+      String sqlDate = (String) row.get("SQLDATE");
+      String actor1Name = (String) row.get("Actor1Name");
+      String sourceUrl = (String) row.get("SOURCEURL");
+      String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
+      c.output(KV.of(countryCode, eventInfo));
+    }
+  }
+
+
+  /**
+   * Examines each row (country info) in the input table. Output a KV with the key the country
+   * code, and the value the country name.
+   */
+  static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = c.element();
+      String countryCode = (String) row.get("FIPSCC");
+      String countryName = (String) row.get("HumanName");
+      c.output(KV.of(countryCode, countryName));
+    }
+  }
+
+
+  /**
+   * Options supported by {@link JoinExamples}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Path of the file to write to")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  public static void main(String[] args) throws Exception {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+    // the following two 'applys' create multiple inputs to our pipeline, one for each
+    // of our two input sources.
+    PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
+    PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
+    PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
+    formattedResults.apply(TextIO.Write.to(options.getOutput()));
+    p.run();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
new file mode 100644
index 0000000..b35a57f
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
@@ -0,0 +1,174 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Max;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An example that reads the public samples of weather data from BigQuery, and finds
+ * the maximum temperature ('mean_temp') for each month.
+ *
+ * <p>Concepts: The 'Max' statistical combination function, and how to find the max per
+ * key group.
+ *
+ * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
+ * table.
+ *
+ * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * </pre>
+ * and the BigQuery table for the output, with the form
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * }
+ * </pre>
+ * and the BigQuery table for the output:
+ * <pre>{@code
+ *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ * }</pre>
+ *
+ * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations }
+ * and can be overridden with {@code --input}.
+ */
+public class MaxPerKeyExamples {
+  // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod.
+  private static final String WEATHER_SAMPLES_TABLE =
+      "clouddataflow-readonly:samples.weather_stations";
+
+  /**
+   * Examines each row (weather reading) in the input table. Output the month of the reading,
+   * and the mean_temp.
+   */
+  static class ExtractTempFn extends DoFn<TableRow, KV<Integer, Double>> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = c.element();
+      Integer month = Integer.parseInt((String) row.get("month"));
+      Double meanTemp = Double.parseDouble(row.get("mean_temp").toString());
+      c.output(KV.of(month, meanTemp));
+    }
+  }
+
+  /**
+   * Format the results to a TableRow, to save to BigQuery.
+   *
+   */
+  static class FormatMaxesFn extends DoFn<KV<Integer, Double>, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      TableRow row = new TableRow()
+          .set("month", c.element().getKey())
+          .set("max_mean_temp", c.element().getValue());
+      c.output(row);
+    }
+  }
+
+  /**
+   * Reads rows from a weather data table, and finds the max mean_temp for each
+   * month via the 'Max' statistical combination function.
+   */
+  static class MaxMeanTemp
+      extends PTransform<PCollection<TableRow>, PCollection<TableRow>> {
+    @Override
+    public PCollection<TableRow> apply(PCollection<TableRow> rows) {
+
+      // row... => <month, mean_temp> ...
+      PCollection<KV<Integer, Double>> temps = rows.apply(
+          ParDo.of(new ExtractTempFn()));
+
+      // month, mean_temp... => <month, max mean temp>...
+      PCollection<KV<Integer, Double>> tempMaxes =
+          temps.apply(Max.<Integer>doublesPerKey());
+
+      // <month, max>... => row...
+      PCollection<TableRow> results = tempMaxes.apply(
+          ParDo.of(new FormatMaxesFn()));
+
+      return results;
+    }
+  }
+
+  /**
+   * Options supported by {@link MaxPerKeyExamples}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private static interface Options extends PipelineOptions {
+    @Description("Table to read from, specified as "
+        + "<project_id>:<dataset_id>.<table_id>")
+    @Default.String(WEATHER_SAMPLES_TABLE)
+    String getInput();
+    void setInput(String value);
+
+    @Description("Table to write to, specified as "
+        + "<project_id>:<dataset_id>.<table_id>")
+    @Validation.Required
+    String getOutput();
+    void setOutput(String value);
+  }
+
+  public static void main(String[] args)
+      throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+
+    // Build the table schema for the output table.
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("month").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("max_mean_temp").setType("FLOAT"));
+    TableSchema schema = new TableSchema().setFields(fields);
+
+    p.apply(BigQueryIO.Read.from(options.getInput()))
+     .apply(new MaxMeanTemp())
+     .apply(BigQueryIO.Write
+        .to(options.getOutput())
+        .withSchema(schema)
+        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
new file mode 100644
index 0000000..99f3080
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
@@ -0,0 +1,55 @@
+
+# "Cookbook" Examples
+
+This directory holds simple "cookbook" examples, which show how to define
+commonly-used data analysis patterns that you would likely incorporate into a
+larger Dataflow pipeline. They include:
+
+ <ul>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java">BigQueryTornadoes</a>
+  &mdash; An example that reads the public samples of weather data from Google
+  BigQuery, counts the number of tornadoes that occur in each month, and
+  writes the results to BigQuery. Demonstrates reading/writing BigQuery,
+  counting a <code>PCollection</code>, and user-defined <code>PTransforms</code>.</li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java">CombinePerKeyExamples</a>
+  &mdash; An example that reads the public &quot;Shakespeare&quot; data, and for
+  each word in the dataset that exceeds a given length, generates a string
+  containing the list of play names in which that word appears.
+  Demonstrates the <code>Combine.perKey</code>
+  transform, which lets you combine the values in a key-grouped
+  <code>PCollection</code>.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java">DatastoreWordCount</a>
+  &mdash; An example that shows you how to read from Google Cloud Datastore.</li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java">DeDupExample</a>
+  &mdash; An example that uses Shakespeare's plays as plain text files, and
+  removes duplicate lines across all the files. Demonstrates the
+  <code>RemoveDuplicates</code>, <code>TextIO.Read</code>,
+  and <code>TextIO.Write</code> transforms, and how to wire transforms together.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java">FilterExamples</a>
+  &mdash; An example that shows different approaches to filtering, including
+  selection and projection. It also shows how to dynamically set parameters
+  by defining and using new pipeline options, and use how to use a value derived
+  by a pipeline. Demonstrates the <code>Mean</code> transform,
+  <code>Options</code> configuration, and using pipeline-derived data as a side
+  input.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java">JoinExamples</a>
+  &mdash; An example that shows how to join two collections. It uses a
+  sample of the <a href="http://goo.gl/OB6oin">GDELT &quot;world event&quot;
+  data</a>, joining the event <code>action</code> country code against a table
+  that maps country codes to country names. Demonstrates the <code>Join</code>
+  operation, and using multiple input sources.
+  </li>
+  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java">MaxPerKeyExamples</a>
+  &mdash; An example that reads the public samples of weather data from BigQuery,
+  and finds the maximum temperature (<code>mean_temp</code>) for each month.
+  Demonstrates the <code>Max</code> statistical combination transform, and how to
+  find the max-per-key group.
+  </li>
+  </ul>
+
+See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
+README](../../../../../../../../../README.md) for
+information about how to run these examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/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
new file mode 100644
index 0000000..e32596d
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -0,0 +1,565 @@
+/*
+ * 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 com.google.cloud.dataflow.examples.cookbook;
+
+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.cloud.dataflow.examples.common.DataflowExampleOptions;
+import com.google.cloud.dataflow.examples.common.DataflowExampleUtils;
+import com.google.cloud.dataflow.examples.common.ExampleBigQueryTableOptions;
+import com.google.cloud.dataflow.examples.common.ExamplePubsubTopicOptions;
+import com.google.cloud.dataflow.examples.common.PubsubFileInjector;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.PubsubIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.IntraBundleParallelization;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This example illustrates the basic concepts behind triggering. It shows how to use different
+ * trigger definitions to produce partial (speculative) results before all the data is processed and
+ * to control when updated results are produced for late data. The example performs a streaming
+ * analysis of the data coming in from PubSub and writes the results to BigQuery. It divides the
+ * data into {@link Window windows} to be processed, and demonstrates using various kinds of {@link
+ * Trigger triggers} to control when the results for each window are emitted.
+ *
+ * <p> This example uses a portion of real traffic data from San Diego freeways. It contains
+ * readings from sensor stations set up along each freeway. Each sensor reading includes a
+ * calculation of the 'total flow' across all lanes in that freeway direction.
+ *
+ * <p> Concepts:
+ * <pre>
+ *   1. The default triggering behavior
+ *   2. Late data with the default trigger
+ *   3. How to get speculative estimates
+ *   4. Combining late data and speculative estimates
+ * </pre>
+ *
+ * <p> Before running this example, it will be useful to familiarize yourself with Dataflow triggers
+ * and understand the concept of 'late data',
+ * See:  <a href="https://cloud.google.com/dataflow/model/triggers">
+ * https://cloud.google.com/dataflow/model/triggers </a> and
+ * <a href="https://cloud.google.com/dataflow/model/windowing#Advanced">
+ * https://cloud.google.com/dataflow/model/windowing#Advanced </a>
+ *
+ * <p> The example pipeline reads data from a Pub/Sub topic. By default, running the example will
+ * also run an auxiliary pipeline to inject data from the default {@code --input} file to the
+ * {@code --pubsubTopic}. The auxiliary pipeline puts a timestamp on the injected data so that the
+ * example pipeline can operate on <i>event time</i> (rather than arrival time). The auxiliary
+ * pipeline also randomly simulates late data, by setting the timestamps of some of the data
+ * elements to be in the past. You may override the default {@code --input} with the file of your
+ * choosing or set {@code --input=""} which will disable the automatic Pub/Sub injection, and allow
+ * you to use a separate tool to publish to the given topic.
+ *
+ * <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 {@code --pubsubTopic}, {@code --bigQueryDataset}, and
+ * {@code --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
+ * the example will try to create them.
+ *
+ * <p> The pipeline outputs its results to a BigQuery table.
+ * Here are some queries you can use to see interesting results:
+ * Replace {@code <enter_table_name>} in the query below with the name of the BigQuery table.
+ * Replace {@code <enter_window_interval>} in the query below with the window interval.
+ *
+ * <p> To see the results of the default trigger,
+ * Note: When you start up your pipeline, you'll initially see results from 'late' data. Wait after
+ * the window duration, until the first pane of non-late data has been emitted, to see more
+ * interesting results.
+ * {@code SELECT * FROM enter_table_name WHERE trigger_type = "default" ORDER BY window DESC}
+ *
+ * <p> To see the late data i.e. dropped by the default trigger,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "withAllowedLateness" and
+ * (timing = "LATE" or timing = "ON_TIME") and freeway = "5" ORDER BY window DESC, processing_time}
+ *
+ * <p>To see the the difference between accumulation mode and discarding mode,
+ * {@code SELECT * FROM <enter_table_name> WHERE (timing = "LATE" or timing = "ON_TIME") AND
+ * (trigger_type = "withAllowedLateness" or trigger_type = "sequential") and freeway = "5" ORDER BY
+ * window DESC, processing_time}
+ *
+ * <p> To see speculative results every minute,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "speculative" and freeway = "5"
+ * ORDER BY window DESC, processing_time}
+ *
+ * <p> To see speculative results every five minutes after the end of the window
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "sequential" and timing != "EARLY"
+ * and freeway = "5" ORDER BY window DESC, processing_time}
+ *
+ * <p> To see the first and the last pane for a freeway in a window for all the trigger types,
+ * {@code SELECT * FROM <enter_table_name> WHERE (isFirst = true or isLast = true) ORDER BY window}
+ *
+ * <p> To reduce the number of results for each query we can add additional where clauses.
+ * For examples, To see the results of the default trigger,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "default" AND freeway = "5" AND
+ * window = "<enter_window_interval>"}
+ *
+ * <p> The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
+ * and then exits.
+ */
+
+public class TriggerExample {
+  //Numeric value of fixed window duration, in minutes
+  public static final int WINDOW_DURATION = 30;
+  // Constants used in triggers.
+  // Speeding up ONE_MINUTE or FIVE_MINUTES helps you get an early approximation of results.
+  // ONE_MINUTE is used only with processing time before the end of the window
+  public static final Duration ONE_MINUTE = Duration.standardMinutes(1);
+  // FIVE_MINUTES is used only with processing time after the end of the window
+  public static final Duration FIVE_MINUTES = Duration.standardMinutes(5);
+  // ONE_DAY is used to specify the amount of lateness allowed for the data elements.
+  public static final Duration ONE_DAY = Duration.standardDays(1);
+
+  /**
+   * This transform demonstrates using triggers to control when data is produced for each window
+   * Consider an example to understand the results generated by each type of trigger.
+   * The example uses "freeway" as the key. Event time is the timestamp associated with the data
+   * element and processing time is the time when the data element gets processed in the pipeline.
+   * For freeway 5, suppose there are 10 elements in the [10:00:00, 10:30:00) window.
+   * Key (freeway) | Value (total_flow) | event time | processing time
+   * 5             | 50                 | 10:00:03   | 10:00:47
+   * 5             | 30                 | 10:01:00   | 10:01:03
+   * 5             | 30                 | 10:02:00   | 11:07:00
+   * 5             | 20                 | 10:04:10   | 10:05:15
+   * 5             | 60                 | 10:05:00   | 11:03:00
+   * 5             | 20                 | 10:05:01   | 11.07:30
+   * 5             | 60                 | 10:15:00   | 10:27:15
+   * 5             | 40                 | 10:26:40   | 10:26:43
+   * 5             | 60                 | 10:27:20   | 10:27:25
+   * 5             | 60                 | 10:29:00   | 11:11:00
+   *
+   * <p> Dataflow tracks a watermark which records up to what point in event time the data is
+   * complete. For the purposes of the example, we'll assume the watermark is approximately 15m
+   * behind the current processing time. In practice, the actual value would vary over time based
+   * on the systems knowledge of the current PubSub delay and contents of the backlog (data
+   * that has not yet been processed).
+   *
+   * <p> If the watermark is 15m behind, then the window [10:00:00, 10:30:00) (in event time) would
+   * close at 10:44:59, when the watermark passes 10:30:00.
+   */
+  static class CalculateTotalFlow
+  extends PTransform <PCollection<KV<String, Integer>>, PCollectionList<TableRow>> {
+    private int windowDuration;
+
+    CalculateTotalFlow(int windowDuration) {
+      this.windowDuration = windowDuration;
+    }
+
+    @Override
+    public PCollectionList<TableRow> apply(PCollection<KV<String, Integer>> flowInfo) {
+
+      // Concept #1: The default triggering behavior
+      // By default Dataflow uses a trigger which fires when the watermark has passed the end of the
+      // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+
+      // The system also defaults to dropping late data -- data which arrives after the watermark
+      // has passed the event timestamp of the arriving element. This means that the default trigger
+      // will only fire once.
+
+      // Each pane produced by the default trigger with no allowed lateness will be the first and
+      // last pane in the window, and will be ON_TIME.
+
+      // The results for the example above with the default trigger and zero allowed lateness
+      // would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 260                | 6                 | true    | true   | ON_TIME
+
+      // At 11:03:00 (processing time) the system watermark may have advanced to 10:54:00. As a
+      // result, when the data record with event time 10:05:00 arrives at 11:03:00, it is considered
+      // late, and dropped.
+
+      PCollection<TableRow> defaultTriggerResults = flowInfo
+          .apply("Default", Window
+              // The default window duration values work well if you're running the default input
+              // file. You may want to adjust the window duration otherwise.
+              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
+              // The default trigger first emits output when the system's watermark passes the end
+              // of the window.
+              .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
+              // Late data is dropped
+              .withAllowedLateness(Duration.ZERO)
+              // Discard elements after emitting each pane.
+              // With no allowed lateness and the specified trigger there will only be a single
+              // pane, so this doesn't have a noticeable effect. See concept 2 for more details.
+              .discardingFiredPanes())
+          .apply(new TotalFlow("default"));
+
+      // Concept #2: Late data with the default trigger
+      // This uses the same trigger as concept #1, but allows data that is up to ONE_DAY late. This
+      // leads to each window staying open for ONE_DAY after the watermark has passed the end of the
+      // window. Any late data will result in an additional pane being fired for that same window.
+
+      // The first pane produced will be ON_TIME and the remaining panes will be LATE.
+      // To definitely get the last pane when the window closes, use
+      // .withAllowedLateness(ONE_DAY, ClosingBehavior.FIRE_ALWAYS).
+
+      // The results for the example above with the default trigger and ONE_DAY allowed lateness
+      // would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 260                | 6                 | true    | false  | ON_TIME
+      // 5             | 60                 | 1                 | false   | false  | LATE
+      // 5             | 30                 | 1                 | false   | false  | LATE
+      // 5             | 20                 | 1                 | false   | false  | LATE
+      // 5             | 60                 | 1                 | false   | false  | LATE
+      PCollection<TableRow> withAllowedLatenessResults = flowInfo
+          .apply("WithLateData", Window
+              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
+              // Late data is emitted as it arrives
+              .triggering(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
+              // Once the output is produced, the pane is dropped and we start preparing the next
+              // pane for the window
+              .discardingFiredPanes()
+              // Late data is handled up to one day
+              .withAllowedLateness(ONE_DAY))
+          .apply(new TotalFlow("withAllowedLateness"));
+
+      // Concept #3: How to get speculative estimates
+      // We can specify a trigger that fires independent of the watermark, for instance after
+      // ONE_MINUTE of processing time. This allows us to produce speculative estimates before
+      // all the data is available. Since we don't have any triggers that depend on the watermark
+      // we don't get an ON_TIME firing. Instead, all panes are either EARLY or LATE.
+
+      // We also use accumulatingFiredPanes to build up the results across each pane firing.
+
+      // The results for the example above for this trigger would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 80                 | 2                 | true    | false  | EARLY
+      // 5             | 100                | 3                 | false   | false  | EARLY
+      // 5             | 260                | 6                 | false   | false  | EARLY
+      // 5             | 320                | 7                 | false   | false  | LATE
+      // 5             | 370                | 9                 | false   | false  | LATE
+      // 5             | 430                | 10                | false   | false  | LATE
+      PCollection<TableRow> speculativeResults = flowInfo
+          .apply("Speculative" , Window
+              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
+              // Trigger fires every minute.
+              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
+                  // Speculative every ONE_MINUTE
+                  .plusDelayOf(ONE_MINUTE)))
+              // After emitting each pane, it will continue accumulating the elements so that each
+              // approximation includes all of the previous data in addition to the newly arrived
+              // data.
+              .accumulatingFiredPanes()
+              .withAllowedLateness(ONE_DAY))
+          .apply(new TotalFlow("speculative"));
+
+      // Concept #4: Combining late data and speculative estimates
+      // We can put the previous concepts together to get EARLY estimates, an ON_TIME result,
+      // and LATE updates based on late data.
+
+      // Each time a triggering condition is satisfied it advances to the next trigger.
+      // If there are new elements this trigger emits a window under following condition:
+      // > Early approximations every minute till the end of the window.
+      // > An on-time firing when the watermark has passed the end of the window
+      // > Every five minutes of late data.
+
+      // Every pane produced will either be EARLY, ON_TIME or LATE.
+
+      // The results for the example above for this trigger would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 80                 | 2                 | true    | false  | EARLY
+      // 5             | 100                | 3                 | false   | false  | EARLY
+      // 5             | 260                | 6                 | false   | false  | EARLY
+      // [First pane fired after the end of the window]
+      // 5             | 320                | 7                 | false   | false  | ON_TIME
+      // 5             | 430                | 10                | false   | false  | LATE
+
+      // For more possibilities of how to build advanced triggers, see {@link Trigger}.
+      PCollection<TableRow> sequentialResults = flowInfo
+          .apply("Sequential", Window
+              .<KV<String, Integer>>into(FixedWindows.of(Duration.standardMinutes(windowDuration)))
+              .triggering(AfterEach.inOrder(
+                  Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
+                      // Speculative every ONE_MINUTE
+                      .plusDelayOf(ONE_MINUTE)).orFinally(AfterWatermark.pastEndOfWindow()),
+                  Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
+                      // Late data every FIVE_MINUTES
+                      .plusDelayOf(FIVE_MINUTES))))
+              .accumulatingFiredPanes()
+              // For up to ONE_DAY
+              .withAllowedLateness(ONE_DAY))
+          .apply(new TotalFlow("sequential"));
+
+      // Adds the results generated by each trigger type to a PCollectionList.
+      PCollectionList<TableRow> resultsList = PCollectionList.of(defaultTriggerResults)
+          .and(withAllowedLatenessResults)
+          .and(speculativeResults)
+          .and(sequentialResults);
+
+      return resultsList;
+    }
+  }
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+  // The remaining parts of the pipeline are needed to produce the output for each
+  // concept above. Not directly relevant to understanding the trigger examples.
+
+  /**
+   * Calculate total flow and number of records for each freeway and format the results to TableRow
+   * objects, to save to BigQuery.
+   */
+  static class TotalFlow extends
+  PTransform <PCollection<KV<String, Integer>>, PCollection<TableRow>> {
+    private String triggerType;
+
+    public TotalFlow(String triggerType) {
+      this.triggerType = triggerType;
+    }
+
+    @Override
+    public PCollection<TableRow> apply(PCollection<KV<String, Integer>> flowInfo) {
+      PCollection<KV<String, Iterable<Integer>>> flowPerFreeway = flowInfo
+          .apply(GroupByKey.<String, Integer>create());
+
+      PCollection<KV<String, String>> results = flowPerFreeway.apply(ParDo.of(
+          new DoFn <KV<String, Iterable<Integer>>, KV<String, String>>() {
+
+            @Override
+            public void processElement(ProcessContext c) throws Exception {
+              Iterable<Integer> flows = c.element().getValue();
+              Integer sum = 0;
+              Long numberOfRecords = 0L;
+              for (Integer value : flows) {
+                sum += value;
+                numberOfRecords++;
+              }
+              c.output(KV.of(c.element().getKey(), sum + "," + numberOfRecords));
+            }
+          }));
+      PCollection<TableRow> output = results.apply(ParDo.of(new FormatTotalFlow(triggerType)));
+      return output;
+    }
+  }
+
+  /**
+   * Format the results of the Total flow calculation to a TableRow, to save to BigQuery.
+   * Adds the triggerType, pane information, processing time and the window timestamp.
+   * */
+  static class FormatTotalFlow extends DoFn<KV<String, String>, TableRow>
+  implements  RequiresWindowAccess {
+    private String triggerType;
+
+    public FormatTotalFlow(String triggerType) {
+      this.triggerType = triggerType;
+    }
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      String[] values = c.element().getValue().split(",");
+      TableRow row = new TableRow()
+          .set("trigger_type", triggerType)
+          .set("freeway", c.element().getKey())
+          .set("total_flow", Integer.parseInt(values[0]))
+          .set("number_of_records", Long.parseLong(values[1]))
+          .set("window", c.window().toString())
+          .set("isFirst", c.pane().isFirst())
+          .set("isLast", c.pane().isLast())
+          .set("timing", c.pane().getTiming().toString())
+          .set("event_time", c.timestamp().toString())
+          .set("processing_time", Instant.now().toString());
+      c.output(row);
+    }
+  }
+
+  /**
+   * Extract the freeway and total flow in a reading.
+   * Freeway is used as key since we are calculating the total flow for each freeway.
+   */
+  static class ExtractFlowInfo extends DoFn<String, KV<String, Integer>> {
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      String[] laneInfo = c.element().split(",");
+      if (laneInfo[0].equals("timestamp")) {
+        // Header row
+        return;
+      }
+      if (laneInfo.length < 48) {
+        //Skip the invalid input.
+        return;
+      }
+      String freeway = laneInfo[2];
+      Integer totalFlow = tryIntegerParse(laneInfo[7]);
+      // Ignore the records with total flow 0 to easily understand the working of triggers.
+      // Skip the records with total flow -1 since they are invalid input.
+      if (totalFlow == null || totalFlow <= 0) {
+        return;
+      }
+      c.output(KV.of(freeway,  totalFlow));
+    }
+  }
+
+  /**
+   * Inherits standard configuration options.
+   */
+  public interface TrafficFlowOptions
+      extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions, DataflowExampleOptions {
+
+    @Description("Input file to inject to Pub/Sub topic")
+    @Default.String("gs://dataflow-samples/traffic_sensor/"
+        + "Freeways-5Minaa2010-01-01_to_2010-02-15.csv")
+    String getInput();
+    void setInput(String value);
+
+    @Description("Numeric value of window duration for fixed windows, in minutes")
+    @Default.Integer(WINDOW_DURATION)
+    Integer getWindowDuration();
+    void setWindowDuration(Integer value);
+  }
+
+  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
+
+  public static void main(String[] args) throws Exception {
+    TrafficFlowOptions options = PipelineOptionsFactory.fromArgs(args)
+        .withValidation()
+        .as(TrafficFlowOptions.class);
+    options.setStreaming(true);
+
+    // In order to cancel the pipelines automatically,
+    // {@code DataflowPipelineRunner} is forced to be used.
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setBigQuerySchema(getSchema());
+
+    DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options);
+    dataflowUtils.setup();
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    TableReference tableRef = getTableReference(options.getProject(),
+        options.getBigQueryDataset(), options.getBigQueryTable());
+
+    PCollectionList<TableRow> resultList = pipeline.apply(PubsubIO.Read.named("ReadPubsubInput")
+        .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
+        .topic(options.getPubsubTopic()))
+        .apply(ParDo.of(new ExtractFlowInfo()))
+        .apply(new CalculateTotalFlow(options.getWindowDuration()));
+
+    for (int i = 0; i < resultList.size(); i++){
+      resultList.get(i).apply(BigQueryIO.Write.to(tableRef).withSchema(getSchema()));
+    }
+
+    PipelineResult result = pipeline.run();
+    if (!options.getInput().isEmpty()){
+      //Inject the data into the pubsub topic
+      dataflowUtils.runInjectorPipeline(runInjector(options));
+    }
+    // dataflowUtils will try to cancel the pipeline and the injector before the program exits.
+    dataflowUtils.waitToFinish(result);
+  }
+
+  private static Pipeline runInjector(TrafficFlowOptions options){
+    DataflowPipelineOptions copiedOptions = options.cloneAs(DataflowPipelineOptions.class);
+    copiedOptions.setStreaming(false);
+    copiedOptions.setNumWorkers(options.as(DataflowExampleOptions.class).getInjectorNumWorkers());
+    copiedOptions.setJobName(options.getJobName() + "-injector");
+    Pipeline injectorPipeline = Pipeline.create(copiedOptions);
+    injectorPipeline
+    .apply(TextIO.Read.named("ReadMyFile").from(options.getInput()))
+    .apply(ParDo.named("InsertRandomDelays").of(new InsertDelays()))
+    .apply(IntraBundleParallelization.of(PubsubFileInjector
+        .withTimestampLabelKey(PUBSUB_TIMESTAMP_LABEL_KEY)
+        .publish(options.getPubsubTopic()))
+        .withMaxParallelism(20));
+
+    return injectorPipeline;
+  }
+
+  /**
+   * Add current time to each record.
+   * Also insert a delay at random to demo the triggers.
+   */
+  public static class InsertDelays extends DoFn<String, String> {
+    private static final double THRESHOLD = 0.001;
+    // MIN_DELAY and MAX_DELAY in minutes.
+    private static final int MIN_DELAY = 1;
+    private static final int MAX_DELAY = 100;
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      Instant timestamp = Instant.now();
+      if (Math.random() < THRESHOLD){
+        int range = MAX_DELAY - MIN_DELAY;
+        int delayInMinutes = (int) (Math.random() * range) + MIN_DELAY;
+        long delayInMillis = TimeUnit.MINUTES.toMillis(delayInMinutes);
+        timestamp = new Instant(timestamp.getMillis() - delayInMillis);
+      }
+      c.outputWithTimestamp(c.element(), timestamp);
+    }
+  }
+
+
+  /**Sets the table reference. **/
+  private static TableReference getTableReference(String project, String dataset, String table){
+    TableReference tableRef = new TableReference();
+    tableRef.setProjectId(project);
+    tableRef.setDatasetId(dataset);
+    tableRef.setTableId(table);
+    return tableRef;
+  }
+
+  /** Defines the BigQuery schema used for the output. */
+  private static TableSchema getSchema() {
+    List<TableFieldSchema> fields = new ArrayList<>();
+    fields.add(new TableFieldSchema().setName("trigger_type").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("freeway").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("total_flow").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("number_of_records").setType("INTEGER"));
+    fields.add(new TableFieldSchema().setName("window").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("isFirst").setType("BOOLEAN"));
+    fields.add(new TableFieldSchema().setName("isLast").setType("BOOLEAN"));
+    fields.add(new TableFieldSchema().setName("timing").setType("STRING"));
+    fields.add(new TableFieldSchema().setName("event_time").setType("TIMESTAMP"));
+    fields.add(new TableFieldSchema().setName("processing_time").setType("TIMESTAMP"));
+    TableSchema schema = new TableSchema().setFields(fields);
+    return schema;
+  }
+
+  private static Integer tryIntegerParse(String number) {
+    try {
+      return Integer.parseInt(number);
+    } catch (NumberFormatException e) {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/examples/java/src/test/java/com/google/cloud/dataflow/examples/DebuggingWordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/com/google/cloud/dataflow/examples/DebuggingWordCountTest.java b/examples/java/src/test/java/com/google/cloud/dataflow/examples/DebuggingWordCountTest.java
deleted file mode 100644
index 52c61ef..0000000
--- a/examples/java/src/test/java/com/google/cloud/dataflow/examples/DebuggingWordCountTest.java
+++ /dev/null
@@ -1,46 +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 com.google.cloud.dataflow.examples;
-
-import com.google.common.io.Files;
-
-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;
-import java.nio.charset.StandardCharsets;
-
-/**
- * Tests for {@link DebuggingWordCount}.
- */
-@RunWith(JUnit4.class)
-public class DebuggingWordCountTest {
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Test
-  public void testDebuggingWordCount() throws Exception {
-    File file = tmpFolder.newFile();
-    Files.write("stomach secret Flourish message Flourish here Flourish", file,
-        StandardCharsets.UTF_8);
-    DebuggingWordCount.main(new String[]{"--inputFile=" + file.getAbsolutePath()});
-  }
-}
-



[65/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 da9e0a0..3e4c6d2 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
@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode;
+import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException.ReasonCode;
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 728f761..9f997c0 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
 import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java
index a43e95e..69412e5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java
@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addStringList;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.StringUtils;
+
 import com.google.common.collect.Lists;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
index b277166..3eb1253 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java
index d1df083..905178b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
index 6dc2506..8998ea5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 /**
  * A {@link DeterministicStandardCoder} is a {@link StandardCoder} that is

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java
index 44cade6..cb204ec 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java
index 46fbdde..9db6650 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/EntityCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/EntityCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/EntityCoder.java
index ffd516a..a704772 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/EntityCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/EntityCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.google.api.services.datastore.DatastoreV1.Entity;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java
index 65c517f..0be6ef8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
 import com.google.common.base.Converter;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 55843ee..546a240 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
 import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 194e9ef..74e3dd3 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
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+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.cloud.dataflow.sdk.util.BufferedElementCountingOutputStream;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterable;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
 import com.google.common.base.Preconditions;
 
 import java.io.DataInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java
index 6e2833e..6fc8fcf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.util.VarInt;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.util.VarInt;
 import com.google.common.io.ByteStreams;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 227c546..6020d2c 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
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+
+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.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
 import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoderBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoderBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoderBase.java
index 361ffb9..cf65bcd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoderBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoderBase.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 cfb3ab7..c45559a 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
 import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 5a60ab5..1e56eec 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
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoderBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoderBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoderBase.java
index 1fc3f57..b808ca2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoderBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoderBase.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 fc12a03..6fa7305 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
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Proto2Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Proto2Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Proto2Coder.java
index 1ce5fe5..20bc821 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Proto2Coder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Proto2Coder.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
index 1590b11..310ecb7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 589a372..d194a53 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
 import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java
index ca189b1..423d482 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java
@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import static org.apache.beam.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addStringList;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
 import com.google.common.collect.Lists;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.CountingOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
index 3f352d3..0e62311 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
+import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
 
 import java.lang.reflect.InvocationTargetException;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java
index 25b8f5e..807c001 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.util.ExposedByteArrayOutputStream;
+import org.apache.beam.sdk.util.StreamUtils;
+import org.apache.beam.sdk.util.VarInt;
 
-import com.google.cloud.dataflow.sdk.util.ExposedByteArrayOutputStream;
-import com.google.cloud.dataflow.sdk.util.StreamUtils;
-import com.google.cloud.dataflow.sdk.util.VarInt;
 import com.google.common.base.Utf8;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.CountingOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java
index aa44456..226f79c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static com.google.api.client.util.Base64.encodeBase64String;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java
index b02fb08..42a6dc9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.google.api.services.bigquery.model.TableRow;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java
index 539c56a..156160e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java
index 42862bb..20ce7d4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.VarInt;
+import org.apache.beam.sdk.util.VarInt;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java
index 669453e..f1a5ea0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.util.VarInt;
+import org.apache.beam.sdk.util.VarInt;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java
index 813ee2f..d67d82f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
index a3bc150..c5a90e0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java
@@ -16,30 +16,30 @@
  * limitations under the License.
  */
 /**
- * Defines {@link com.google.cloud.dataflow.sdk.coders.Coder Coders}
+ * Defines {@link org.apache.beam.sdk.coders.Coder Coders}
  * to specify how data is encoded to and decoded from byte strings.
  *
  * <p>During execution of a Pipeline, elements in a
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}
+ * {@link org.apache.beam.sdk.values.PCollection}
  * may need to be encoded into byte strings.
  * This happens both at the beginning and end of a pipeline when data is read from and written to
  * persistent storage and also during execution of a pipeline when elements are communicated between
  * machines.
  *
  * <p>Exactly when PCollection elements are encoded during execution depends on which
- * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is being used and how that runner
+ * {@link org.apache.beam.sdk.runners.PipelineRunner} is being used and how that runner
  * chooses to execute the pipeline. As such, Dataflow requires that all PCollections have an
  * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from
  * the available Java type
- * information and the Pipeline's {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry}. It
+ * information and the Pipeline's {@link org.apache.beam.sdk.coders.CoderRegistry}. It
  * can be specified per PCollection via
- * {@link com.google.cloud.dataflow.sdk.values.PCollection#setCoder(Coder)} or per type using the
- * {@link com.google.cloud.dataflow.sdk.coders.DefaultCoder} annotation.
+ * {@link org.apache.beam.sdk.values.PCollection#setCoder(Coder)} or per type using the
+ * {@link org.apache.beam.sdk.coders.DefaultCoder} annotation.
  *
  * <p>This package provides a number of coders for common types like {@code Integer},
  * {@code String}, and {@code List}, as well as coders like
- * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} that can be used to encode many custom
+ * {@link org.apache.beam.sdk.coders.AvroCoder} that can be used to encode many custom
  * types.
  *
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java
index 88d13e9..67cec88 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders.protobuf;
+package org.apache.beam.sdk.coders.protobuf;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.CoderProvider;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.Structs;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderProvider;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java
index 24415ac..b03b748 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders.protobuf;
+package org.apache.beam.sdk.coders.protobuf;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+
 import com.google.protobuf.Descriptors.Descriptor;
 import com.google.protobuf.Descriptors.FieldDescriptor;
 import com.google.protobuf.Descriptors.FileDescriptor.Syntax;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/package-info.java
index 8242fad..bd16484 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/package-info.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 /**
- * Defines a {@link com.google.cloud.dataflow.sdk.coders.Coder}
+ * Defines a {@link org.apache.beam.sdk.coders.Coder}
  * for Protocol Buffers messages, {@code ProtoCoder}.
  *
- * @see com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder
+ * @see org.apache.beam.sdk.coders.protobuf.ProtoCoder
  */
-package com.google.cloud.dataflow.sdk.coders.protobuf;
+package org.apache.beam.sdk.coders.protobuf;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 145804d..8f76ae8 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
@@ -15,22 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.Read.Bounded;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+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.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
@@ -314,9 +315,9 @@ public class AvroIO {
         @SuppressWarnings("unchecked")
         Bounded<T> read =
             type == GenericRecord.class
-                ? (Bounded<T>) com.google.cloud.dataflow.sdk.io.Read.from(
+                ? (Bounded<T>) org.apache.beam.sdk.io.Read.from(
                     AvroSource.from(filepattern).withSchema(schema))
-                : com.google.cloud.dataflow.sdk.io.Read.from(
+                : org.apache.beam.sdk.io.Read.from(
                     AvroSource.from(filepattern).withSchema(type));
 
         PCollection<T> pcol = input.getPipeline().apply("Read", read);
@@ -673,7 +674,7 @@ public class AvroIO {
         // apply user requested sharding limits.
         return input.apply(
             "Write",
-            com.google.cloud.dataflow.sdk.io.Write.to(
+            org.apache.beam.sdk.io.Write.to(
                 new AvroSink<>(
                     filenamePrefix, filenameSuffix, shardTemplate, AvroCoder.of(type, schema))));
       }
@@ -759,7 +760,7 @@ public class AvroIO {
     }
 
     /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation
+     * A {@link org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation
      * FileBasedWriteOperation} for Avro files.
      */
     private static class AvroWriteOperation<T> extends FileBasedWriteOperation<T> {
@@ -777,7 +778,7 @@ public class AvroIO {
     }
 
     /**
-     * A {@link com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
+     * A {@link org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter FileBasedWriter}
      * for Avro files.
      */
     private static class AvroWriter<T> extends FileBasedWriter<T> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 2e017c0..ef8e427 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
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.util.AvroUtils;
-import com.google.cloud.dataflow.sdk.util.AvroUtils.AvroMetadata;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c37d6c7..eb99637 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
@@ -15,11 +15,53 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+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.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.TableRowJsonCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+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.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.BigQueryServices;
+import org.apache.beam.sdk.util.BigQueryServices.LoadService;
+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.MimeTypes;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Reshuffle;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+
 import com.google.api.client.json.JsonFactory;
 import com.google.api.services.bigquery.Bigquery;
 import com.google.api.services.bigquery.model.JobConfigurationLoad;
@@ -27,47 +69,6 @@ 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;
-import com.google.cloud.dataflow.sdk.coders.AtomicCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition;
-import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices;
-import com.google.cloud.dataflow.sdk.util.BigQueryServices.LoadService;
-import com.google.cloud.dataflow.sdk.util.BigQueryServicesImpl;
-import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter;
-import com.google.cloud.dataflow.sdk.util.BigQueryTableRowIterator;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Reshuffle;
-import com.google.cloud.dataflow.sdk.util.SystemDoFnInternal;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
@@ -1004,7 +1005,7 @@ public class BigQueryIO {
         String jobIdToken = UUID.randomUUID().toString();
         String tempFilePrefix = tempLocation + "/BigQuerySinkTemp/" + jobIdToken;
         BigQueryServices bqServices = getBigQueryServices();
-        return input.apply("Write", com.google.cloud.dataflow.sdk.io.Write.to(
+        return input.apply("Write", org.apache.beam.sdk.io.Write.to(
             new BigQuerySink(
                 jobIdToken,
                 jsonTableRef,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
index 86349c9..31ef055 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 
 import java.io.IOException;
 import java.util.NoSuchElementException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
index 2bad6cf..2d36dd1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.ValueWithRecordId;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
 
 import com.google.api.client.util.BackOff;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.IntervalBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.ValueWithRecordId;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java
index 3ed2d7b..8f7d3fd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 import org.joda.time.Instant;
 
@@ -103,7 +103,7 @@ public abstract class BoundedSource<T> extends Source<T> {
    * their interaction is implemented in a thread-safe way, otherwise data loss is possible.
    *
    * <p>Sources which support dynamic work rebalancing should use
-   * {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} to manage the (source-specific)
+   * {@link org.apache.beam.sdk.io.range.RangeTracker} to manage the (source-specific)
    * range of positions that is being split. If your source supports dynamic work rebalancing,
    * please use that class to implement it if possible; if not possible, please contact the team
    * at <i>dataflow-feedback@google.com</i>.
@@ -258,7 +258,7 @@ public abstract class BoundedSource<T> extends Source<T> {
      * {@code synchronized}, because those methods can perform blocking operations, and then
      * this method would have to wait for those calls to complete.
      *
-     * <p>{@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} makes it easy to implement
+     * <p>{@link org.apache.beam.sdk.io.range.RangeTracker} makes it easy to implement
      * this method safely and correctly.
      *
      * <p>By default, returns null to indicate that splitting is not possible.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9543749..84dc960 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+
 import com.google.common.base.Preconditions;
 import com.google.common.io.ByteStreams;
 import com.google.common.primitives.Ints;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java
index 892682d..55e6d96 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.io.CountingSource.NowTimestampFn;
-import com.google.cloud.dataflow.sdk.io.Read.Unbounded;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.io.CountingSource.NowTimestampFn;
+import org.apache.beam.sdk.io.Read.Unbounded;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+
 import com.google.common.base.Optional;
 
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
index 78c98cb..b28e866 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
-import com.google.cloud.dataflow.sdk.io.CountingInput.UnboundedCountingInput;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+
 import com.google.common.collect.ImmutableList;
 
 import org.joda.time.Duration;
@@ -180,7 +181,7 @@ public class CountingSource {
     }
 
     @Override
-    public com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader<Long> createReader(
+    public org.apache.beam.sdk.io.BoundedSource.BoundedReader<Long> createReader(
         PipelineOptions options) throws IOException {
       return new BoundedCountingReader(this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 c2264e1..d65ee6f 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.api.services.datastore.DatastoreV1.PropertyFilter.Operator.EQUAL;
 import static com.google.api.services.datastore.DatastoreV1.PropertyOrder.Direction.DESCENDING;
@@ -29,6 +29,16 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Verify.verify;
 
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.EntityCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+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.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;
@@ -50,16 +60,6 @@ import com.google.api.services.datastore.client.DatastoreFactory;
 import com.google.api.services.datastore.client.DatastoreHelper;
 import com.google.api.services.datastore.client.DatastoreOptions;
 import com.google.api.services.datastore.client.QuerySplitter;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.EntityCoder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.io.Sink.WriteOperation;
-import com.google.cloud.dataflow.sdk.io.Sink.Writer;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -172,7 +172,7 @@ import javax.annotation.Nullable;
  * <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 com.google.cloud.dataflow.sdk.runners.PipelineRunner
+ * @see org.apache.beam.sdk.runners.PipelineRunner
  */
 @Experimental(Experimental.Kind.SOURCE_SINK)
 public class DatastoreIO {
@@ -542,7 +542,7 @@ public class DatastoreIO {
    * {@link Entity Entities} to a Datastore kind.
    *
    */
-  public static class Sink extends com.google.cloud.dataflow.sdk.io.Sink<Entity> {
+  public static class Sink extends org.apache.beam.sdk.io.Sink<Entity> {
     final String host;
     final String datasetId;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 f30960c..391cb96 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
@@ -15,7 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.FileIOChannelFactory;
+import org.apache.beam.sdk.util.GcsIOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
 
 import com.google.api.client.googleapis.batch.BatchRequest;
 import com.google.api.client.googleapis.batch.json.JsonBatchCallback;
@@ -25,26 +46,6 @@ import com.google.api.client.http.HttpRequestInitializer;
 import com.google.api.services.storage.Storage;
 import com.google.api.services.storage.StorageRequest;
 import com.google.api.services.storage.model.StorageObject;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.SerializableCoder;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.FileIOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.cloud.dataflow.sdk.util.MimeTypes;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
index 3ad32b4..5078d6b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.IOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.Futures;
@@ -53,7 +54,7 @@ import java.util.concurrent.Executors;
  *
  * <p>A file-based {@code Source} is a {@code Source} backed by a file pattern defined as a Java
  * glob, a single file, or a offset range for a single file. See {@link OffsetBasedSource} and
- * {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} for semantics of offset ranges.
+ * {@link org.apache.beam.sdk.io.range.RangeTracker} for semantics of offset ranges.
  *
  * <p>This source stores a {@code String} that is an {@link IOChannelFactory} specification for a
  * file or file pattern. There should be an {@code IOChannelFactory} defined for the file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 288688e..9755763 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
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import org.apache.beam.sdk.io.range.OffsetRangeTracker;
+import org.apache.beam.sdk.io.range.RangeTracker;
+import org.apache.beam.sdk.options.PipelineOptions;
 
-import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker;
-import com.google.cloud.dataflow.sdk.io.range.RangeTracker;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 import com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
index e5b8a39..ce5979b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
@@ -16,9 +16,10 @@
  * limitations under the License.
  */
 
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import com.google.api.client.repackaged.com.google.common.base.Preconditions;
+
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java
index 6e34705..66fb61f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java
@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
+
+import org.apache.beam.sdk.options.GcpOptions;
 
 import com.google.api.client.util.DateTime;
 import com.google.auth.oauth2.GoogleCredentials;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.hash.Hashing;
@@ -44,6 +45,7 @@ import com.google.pubsub.v1.ReceivedMessage;
 import com.google.pubsub.v1.SubscriberGrpc;
 import com.google.pubsub.v1.Subscription;
 import com.google.pubsub.v1.Topic;
+
 import io.grpc.Channel;
 import io.grpc.ClientInterceptors;
 import io.grpc.ManagedChannel;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 deed9ab..970fb51 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
@@ -15,11 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import static com.google.common.base.MoreObjects.firstNonNull;
 import static com.google.common.base.Preconditions.checkArgument;
 
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Create;
+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.windowing.AfterWatermark;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+
 import com.google.api.client.util.Clock;
 import com.google.api.client.util.DateTime;
 import com.google.api.services.pubsub.Pubsub;
@@ -30,24 +49,6 @@ import com.google.api.services.pubsub.model.PullRequest;
 import com.google.api.services.pubsub.model.PullResponse;
 import com.google.api.services.pubsub.model.ReceivedMessage;
 import com.google.api.services.pubsub.model.Subscription;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.options.PubsubOptions;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 fb103ee..2427766 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
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
-
-import static com.google.cloud.dataflow.sdk.util.StringUtils.approximateSimpleName;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PInput;
+package org.apache.beam.sdk.io;
+
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PInput;
 
 import org.joda.time.Duration;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ShardNameTemplate.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ShardNameTemplate.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ShardNameTemplate.java
index cc233f1..7f48a5c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ShardNameTemplate.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ShardNameTemplate.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 /**
  * Standard shard naming templates.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
index 7845f47..8f07542 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.PCollection;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
index 86a8e64..1f1cea0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.io;
+package org.apache.beam.sdk.io;
 
 import org.apache.beam.sdk.annotations.Experimental;
-import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.joda.time.Instant;
 


[34/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java
deleted file mode 100644
index e961066..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java
+++ /dev/null
@@ -1,256 +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 com.google.cloud.dataflow.sdk.testing;
-
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowJobExecutionException;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobMessagesHandler;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.base.Optional;
-import com.google.common.base.Throwables;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-/**
- * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
- * {@link DataflowPipelineRunner} when running tests against the {@link TestPipeline}.
- *
- * @see TestPipeline
- */
-public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
-  private static final String TENTATIVE_COUNTER = "tentative";
-  private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
-
-  private final TestDataflowPipelineOptions options;
-  private final DataflowPipelineRunner runner;
-  private int expectedNumberOfAssertions = 0;
-
-  TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
-    this.options = options;
-    this.runner = DataflowPipelineRunner.fromOptions(options);
-  }
-
-  /**
-   * Constructs a runner from the provided options.
-   */
-  public static TestDataflowPipelineRunner fromOptions(
-      PipelineOptions options) {
-    TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
-
-    return new TestDataflowPipelineRunner(dataflowOptions);
-  }
-
-  @Override
-  public DataflowPipelineJob run(Pipeline pipeline) {
-    return run(pipeline, runner);
-  }
-
-  DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
-
-    final DataflowPipelineJob job;
-    try {
-      job = runner.run(pipeline);
-    } catch (DataflowJobExecutionException ex) {
-      throw new IllegalStateException("The dataflow failed.");
-    }
-
-    LOG.info("Running Dataflow job {} with {} expected assertions.",
-        job.getJobId(), expectedNumberOfAssertions);
-
-    CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
-        job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
-
-    try {
-      final Optional<Boolean> result;
-
-      if (options.isStreaming()) {
-        Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
-            new Callable<Optional<Boolean>>() {
-          @Override
-          public Optional<Boolean> call() throws Exception {
-            try {
-              for (;;) {
-                Optional<Boolean> result = checkForSuccess(job);
-                if (result.isPresent()) {
-                  return result;
-                }
-                Thread.sleep(10000L);
-              }
-            } finally {
-              LOG.info("Cancelling Dataflow job {}", job.getJobId());
-              job.cancel();
-            }
-          }
-        });
-        State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
-        if (finalState == null || finalState == State.RUNNING) {
-          LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
-              job.getJobId());
-          job.cancel();
-        }
-        result = resultFuture.get();
-      } else {
-        job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
-        result = checkForSuccess(job);
-      }
-      if (!result.isPresent()) {
-        throw new IllegalStateException(
-            "The dataflow did not output a success or failure metric.");
-      } else if (!result.get()) {
-        throw new AssertionError(messageHandler.getErrorMessage() == null ?
-            "The dataflow did not return a failure reason."
-            : messageHandler.getErrorMessage());
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new RuntimeException(e);
-    } catch (ExecutionException e) {
-      Throwables.propagateIfPossible(e.getCause());
-      throw new RuntimeException(e.getCause());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return job;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    if (transform instanceof PAssert.OneSideInputAssert
-        || transform instanceof PAssert.TwoSideInputAssert) {
-      expectedNumberOfAssertions += 1;
-    }
-
-    return runner.apply(transform, input);
-  }
-
-  Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
-      throws IOException {
-    State state = job.getState();
-    if (state == State.FAILED || state == State.CANCELLED) {
-      LOG.info("The pipeline failed");
-      return Optional.of(false);
-    }
-
-    JobMetrics metrics = job.getDataflowClient().projects().jobs()
-        .getMetrics(job.getProjectId(), job.getJobId()).execute();
-
-    if (metrics == null || metrics.getMetrics() == null) {
-      LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
-    } else {
-      int successes = 0;
-      int failures = 0;
-      for (MetricUpdate metric : metrics.getMetrics()) {
-        if (metric.getName() == null || metric.getName().getContext() == null
-            || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
-          // Don't double count using the non-tentative version of the metric.
-          continue;
-        }
-        if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
-          successes += ((BigDecimal) metric.getScalar()).intValue();
-        } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
-          failures += ((BigDecimal) metric.getScalar()).intValue();
-        }
-      }
-
-      if (failures > 0) {
-        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
-            + "{} expected assertions.", job.getJobId(), successes, failures,
-            expectedNumberOfAssertions);
-        return Optional.of(false);
-      } else if (successes >= expectedNumberOfAssertions) {
-        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
-            + "{} expected assertions.", job.getJobId(), successes, failures,
-            expectedNumberOfAssertions);
-        return Optional.of(true);
-      }
-
-      LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
-          + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
-    }
-
-    return Optional.<Boolean>absent();
-  }
-
-  @Override
-  public String toString() {
-    return "TestDataflowPipelineRunner#" + options.getAppName();
-  }
-
-  /**
-   * Cancels the workflow on the first error message it sees.
-   *
-   * <p>Creates an error message representing the concatenation of all error messages seen.
-   */
-  private static class CancelWorkflowOnError implements JobMessagesHandler {
-    private final DataflowPipelineJob job;
-    private final JobMessagesHandler messageHandler;
-    private final StringBuffer errorMessage;
-    private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
-      this.job = job;
-      this.messageHandler = messageHandler;
-      this.errorMessage = new StringBuffer();
-    }
-
-    @Override
-    public void process(List<JobMessage> messages) {
-      messageHandler.process(messages);
-      for (JobMessage message : messages) {
-        if (message.getMessageImportance() != null
-            && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
-          LOG.info("Dataflow job {} threw exception. Failure message was: {}",
-              job.getJobId(), message.getMessageText());
-          errorMessage.append(message.getMessageText());
-        }
-      }
-      if (errorMessage.length() > 0) {
-        LOG.info("Cancelling Dataflow job {}", job.getJobId());
-        try {
-          job.cancel();
-        } catch (Exception ignore) {
-          // The TestDataflowPipelineRunner will thrown an AssertionError with the job failure
-          // messages.
-        }
-      }
-    }
-
-    private String getErrorMessage() {
-      return errorMessage.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java
deleted file mode 100644
index aeb864a..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java
+++ /dev/null
@@ -1,98 +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 com.google.cloud.dataflow.sdk.util;
-
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-
-/**
- * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
- */
-public class DataflowPathValidator implements PathValidator {
-
-  private DataflowPipelineOptions dataflowOptions;
-
-  DataflowPathValidator(DataflowPipelineOptions options) {
-    this.dataflowOptions = options;
-  }
-
-  public static DataflowPathValidator fromOptions(PipelineOptions options) {
-    return new DataflowPathValidator(options.as(DataflowPipelineOptions.class));
-  }
-
-  /**
-   * Validates the the input GCS path is accessible and that the path
-   * is well formed.
-   */
-  @Override
-  public String validateInputFilePatternSupported(String filepattern) {
-    GcsPath gcsPath = getGcsPath(filepattern);
-    Preconditions.checkArgument(
-        dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
-    String returnValue = verifyPath(filepattern);
-    verifyPathIsAccessible(filepattern, "Could not find file %s");
-    return returnValue;
-  }
-
-  /**
-   * Validates the the output GCS path is accessible and that the path
-   * is well formed.
-   */
-  @Override
-  public String validateOutputFilePrefixSupported(String filePrefix) {
-    String returnValue = verifyPath(filePrefix);
-    verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
-    return returnValue;
-  }
-
-  @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("//"),
-        "Dataflow Service does not allow objects with consecutive slashes");
-    return gcsPath.toResourceName();
-  }
-
-  private void verifyPathIsAccessible(String path, String errorMessage) {
-    GcsPath gcsPath = getGcsPath(path);
-    try {
-      Preconditions.checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
-        errorMessage, path);
-    } catch (IOException e) {
-      throw new RuntimeException(
-          String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
-          e);
-    }
-  }
-
-  private GcsPath getGcsPath(String path) {
-    try {
-      return GcsPath.fromUri(path);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(String.format(
-          "%s expected a valid 'gs://' path but was given '%s'",
-          dataflowOptions.getRunner().getSimpleName(), path), e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowTransport.java
deleted file mode 100644
index 18e6654..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowTransport.java
+++ /dev/null
@@ -1,112 +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 com.google.cloud.dataflow.sdk.util;
-
-import static com.google.cloud.dataflow.sdk.util.Transport.getJsonFactory;
-import static com.google.cloud.dataflow.sdk.util.Transport.getTransport;
-
-import com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.services.clouddebugger.v2.Clouddebugger;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
-import com.google.common.collect.ImmutableList;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-
-/**
- * Helpers for cloud communication.
- */
-public class DataflowTransport {
-
-
-  private static class ApiComponents {
-    public String rootUrl;
-    public String servicePath;
-
-    public ApiComponents(String root, String path) {
-      this.rootUrl = root;
-      this.servicePath = path;
-    }
-  }
-
-  private static ApiComponents apiComponentsFromUrl(String urlString) {
-    try {
-      URL url = new URL(urlString);
-      String rootUrl = url.getProtocol() + "://" + url.getHost() +
-          (url.getPort() > 0 ? ":" + url.getPort() : "");
-      return new ApiComponents(rootUrl, url.getPath());
-    } catch (MalformedURLException e) {
-      throw new RuntimeException("Invalid URL: " + urlString);
-    }
-  }
-
-  /**
-   * Returns a Google Cloud Dataflow client builder.
-   */
-  public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options) {
-    String servicePath = options.getDataflowEndpoint();
-    ApiComponents components;
-    if (servicePath.contains("://")) {
-      components = apiComponentsFromUrl(servicePath);
-    } else {
-      components = new ApiComponents(options.getApiRootUrl(), servicePath);
-    }
-
-    return new Dataflow.Builder(getTransport(),
-        getJsonFactory(),
-        chainHttpRequestInitializer(
-            options.getGcpCredential(),
-            // Do not log 404. It clutters the output and is possibly even required by the caller.
-            new RetryHttpRequestInitializer(ImmutableList.of(404))))
-        .setApplicationName(options.getAppName())
-        .setRootUrl(components.rootUrl)
-        .setServicePath(components.servicePath)
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  public static Clouddebugger.Builder newClouddebuggerClient(DataflowPipelineOptions options) {
-    return new Clouddebugger.Builder(getTransport(),
-        getJsonFactory(),
-        chainHttpRequestInitializer(options.getGcpCredential(), new RetryHttpRequestInitializer()))
-        .setApplicationName(options.getAppName())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  /**
-   * Returns a Dataflow client that does not automatically retry failed
-   * requests.
-   */
-  public static Dataflow.Builder
-      newRawDataflowClient(DataflowPipelineOptions options) {
-    return newDataflowClient(options)
-        .setHttpRequestInitializer(options.getGcpCredential())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
-  private static HttpRequestInitializer chainHttpRequestInitializer(
-      Credential credential, HttpRequestInitializer httpRequestInitializer) {
-    if (credential == null) {
-      return httpRequestInitializer;
-    } else {
-      return new ChainingHttpRequestInitializer(credential, httpRequestInitializer);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java
deleted file mode 100644
index 7307e83..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java
+++ /dev/null
@@ -1,54 +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 com.google.cloud.dataflow.sdk.util;
-
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-
-import java.util.List;
-
-/**
- * Utility class for staging files to GCS.
- */
-public class GcsStager implements Stager {
-  private DataflowPipelineOptions options;
-
-  private GcsStager(DataflowPipelineOptions options) {
-    this.options = options;
-  }
-
-  public static GcsStager fromOptions(PipelineOptions options) {
-    return new GcsStager(options.as(DataflowPipelineOptions.class));
-  }
-
-  @Override
-  public List<DataflowPackage> stageFiles() {
-    Preconditions.checkNotNull(options.getStagingLocation());
-    List<String> filesToStage = options.getFilesToStage();
-    String windmillBinary =
-        options.as(DataflowPipelineDebugOptions.class).getOverrideWindmillBinary();
-    if (windmillBinary != null) {
-      filesToStage.add("windmill_main=" + windmillBinary);
-    }
-    return PackageUtil.stageClasspathElements(
-        options.getFilesToStage(), options.getStagingLocation());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java
deleted file mode 100644
index 2c06a92..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java
+++ /dev/null
@@ -1,235 +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 com.google.cloud.dataflow.sdk.util;
-
-import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * A helper class for monitoring jobs submitted to the service.
- */
-public final class MonitoringUtil {
-
-  private static final String GCLOUD_DATAFLOW_PREFIX = "gcloud alpha dataflow";
-  private static final String ENDPOINT_OVERRIDE_ENV_VAR =
-      "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW";
-
-  private static final Map<String, State> DATAFLOW_STATE_TO_JOB_STATE =
-      ImmutableMap
-          .<String, State>builder()
-          .put("JOB_STATE_UNKNOWN", State.UNKNOWN)
-          .put("JOB_STATE_STOPPED", State.STOPPED)
-          .put("JOB_STATE_RUNNING", State.RUNNING)
-          .put("JOB_STATE_DONE", State.DONE)
-          .put("JOB_STATE_FAILED", State.FAILED)
-          .put("JOB_STATE_CANCELLED", State.CANCELLED)
-          .put("JOB_STATE_UPDATED", State.UPDATED)
-          .build();
-
-  private String projectId;
-  private Messages messagesClient;
-
-  /**
-   * An interface that can be used for defining callbacks to receive a list
-   * of JobMessages containing monitoring information.
-   */
-  public interface JobMessagesHandler {
-    /** Process the rows. */
-    void process(List<JobMessage> messages);
-  }
-
-  /** A handler that prints monitoring messages to a stream. */
-  public static class PrintHandler implements JobMessagesHandler {
-    private PrintStream out;
-
-    /**
-     * Construct the handler.
-     *
-     * @param stream The stream to write the messages to.
-     */
-    public PrintHandler(PrintStream stream) {
-      out = stream;
-    }
-
-    @Override
-    public void process(List<JobMessage> messages) {
-      for (JobMessage message : messages) {
-        if (message.getMessageText() == null || message.getMessageText().isEmpty()) {
-          continue;
-        }
-        String importanceString = null;
-        if (message.getMessageImportance() == null) {
-          continue;
-        } else if (message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
-          importanceString = "Error:   ";
-        } else if (message.getMessageImportance().equals("JOB_MESSAGE_WARNING")) {
-          importanceString = "Warning: ";
-        } else if (message.getMessageImportance().equals("JOB_MESSAGE_BASIC")) {
-          importanceString = "Basic:  ";
-        } else if (message.getMessageImportance().equals("JOB_MESSAGE_DETAILED")) {
-          importanceString = "Detail:  ";
-        } else {
-          // TODO: Remove filtering here once getJobMessages supports minimum
-          // importance.
-          continue;
-        }
-        @Nullable Instant time = TimeUtil.fromCloudTime(message.getTime());
-        if (time == null) {
-          out.print("UNKNOWN TIMESTAMP: ");
-        } else {
-          out.print(time + ": ");
-        }
-        if (importanceString != null) {
-          out.print(importanceString);
-        }
-        out.println(message.getMessageText());
-      }
-      out.flush();
-    }
-  }
-
-  /** Construct a helper for monitoring. */
-  public MonitoringUtil(String projectId, Dataflow dataflow) {
-    this(projectId, dataflow.projects().jobs().messages());
-  }
-
-  // @VisibleForTesting
-  MonitoringUtil(String projectId, Messages messagesClient) {
-    this.projectId = projectId;
-    this.messagesClient = messagesClient;
-  }
-
-  /**
-   * Comparator for sorting rows in increasing order based on timestamp.
-   */
-  public static class TimeStampComparator implements Comparator<JobMessage> {
-    @Override
-    public int compare(JobMessage o1, JobMessage o2) {
-      @Nullable Instant t1 = fromCloudTime(o1.getTime());
-      if (t1 == null) {
-        return -1;
-      }
-      @Nullable Instant t2 = fromCloudTime(o2.getTime());
-      if (t2 == null) {
-        return 1;
-      }
-      return t1.compareTo(t2);
-    }
-  }
-
-  /**
-   * Return job messages sorted in ascending order by timestamp.
-   * @param jobId The id of the job to get the messages for.
-   * @param startTimestampMs Return only those messages with a
-   *   timestamp greater than this value.
-   * @return collection of messages
-   * @throws IOException
-   */
-  public ArrayList<JobMessage> getJobMessages(
-      String jobId, long startTimestampMs) throws IOException {
-    // TODO: Allow filtering messages by importance
-    Instant startTimestamp = new Instant(startTimestampMs);
-    ArrayList<JobMessage> allMessages = new ArrayList<>();
-    String pageToken = null;
-    while (true) {
-      Messages.List listRequest = messagesClient.list(projectId, jobId);
-      if (pageToken != null) {
-        listRequest.setPageToken(pageToken);
-      }
-      ListJobMessagesResponse response = listRequest.execute();
-
-      if (response == null || response.getJobMessages() == null) {
-        return allMessages;
-      }
-
-      for (JobMessage m : response.getJobMessages()) {
-        @Nullable Instant timestamp = fromCloudTime(m.getTime());
-        if (timestamp == null) {
-          continue;
-        }
-        if (timestamp.isAfter(startTimestamp)) {
-          allMessages.add(m);
-        }
-      }
-
-      if (response.getNextPageToken() == null) {
-        break;
-      } else {
-        pageToken = response.getNextPageToken();
-      }
-    }
-
-    Collections.sort(allMessages, new TimeStampComparator());
-    return allMessages;
-  }
-
-  public static String getJobMonitoringPageURL(String projectName, String jobId) {
-    try {
-      // Project name is allowed in place of the project id: the user will be redirected to a URL
-      // that has the project name replaced with project id.
-      return String.format(
-          "https://console.developers.google.com/project/%s/dataflow/job/%s",
-          URLEncoder.encode(projectName, "UTF-8"),
-          URLEncoder.encode(jobId, "UTF-8"));
-    } catch (UnsupportedEncodingException e) {
-      // Should never happen.
-      throw new AssertionError("UTF-8 encoding is not supported by the environment", e);
-    }
-  }
-
-  public static String getGcloudCancelCommand(DataflowPipelineOptions options, String jobId) {
-
-    // If using a different Dataflow API than default, prefix command with an API override.
-    String dataflowApiOverridePrefix = "";
-    String apiUrl = options.getDataflowClient().getBaseUrl();
-    if (!apiUrl.equals(Dataflow.DEFAULT_BASE_URL)) {
-      dataflowApiOverridePrefix = String.format("%s=%s ", ENDPOINT_OVERRIDE_ENV_VAR, apiUrl);
-    }
-
-    // Assemble cancel command from optional prefix and project/job parameters.
-    return String.format("%s%s jobs --project=%s cancel %s",
-        dataflowApiOverridePrefix, GCLOUD_DATAFLOW_PREFIX, options.getProject(), jobId);
-  }
-
-  public static State toState(String stateName) {
-    return MoreObjects.firstNonNull(DATAFLOW_STATE_TO_JOB_STATE.get(stateName),
-        State.UNKNOWN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java
deleted file mode 100644
index 0e234a8..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java
+++ /dev/null
@@ -1,328 +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 com.google.cloud.dataflow.sdk.util;
-
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.hash.Funnels;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
-import com.google.common.io.CountingOutputStream;
-import com.google.common.io.Files;
-
-import com.fasterxml.jackson.core.Base64Variants;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Objects;
-
-/** Helper routines for packages. */
-public class PackageUtil {
-  private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class);
-  /**
-   * A reasonable upper bound on the number of jars required to launch a Dataflow job.
-   */
-  public static final int SANE_CLASSPATH_SIZE = 1000;
-  /**
-   * The initial interval to use between package staging attempts.
-   */
-  private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L;
-  /**
-   * The maximum number of attempts when staging a file.
-   */
-  private static final int MAX_ATTEMPTS = 5;
-
-  /**
-   * Translates exceptions from API calls.
-   */
-  private static final ApiErrorExtractor ERROR_EXTRACTOR = new ApiErrorExtractor();
-
-  /**
-   * Creates a DataflowPackage containing information about how a classpath element should be
-   * staged, including the staging destination as well as its size and hash.
-   *
-   * @param classpathElement The local path for the classpath element.
-   * @param stagingPath The base location for staged classpath elements.
-   * @param overridePackageName If non-null, use the given value as the package name
-   *                            instead of generating one automatically.
-   * @return The package.
-   */
-  @Deprecated
-  public static DataflowPackage createPackage(File classpathElement,
-      String stagingPath, String overridePackageName) {
-    return createPackageAttributes(classpathElement, stagingPath, overridePackageName)
-        .getDataflowPackage();
-  }
-
-  /**
-   * Compute and cache the attributes of a classpath element that we will need to stage it.
-   *
-   * @param classpathElement the file or directory to be staged.
-   * @param stagingPath The base location for staged classpath elements.
-   * @param overridePackageName If non-null, use the given value as the package name
-   *                            instead of generating one automatically.
-   * @return a {@link PackageAttributes} that containing metadata about the object to be staged.
-   */
-  static PackageAttributes createPackageAttributes(File classpathElement,
-      String stagingPath, String overridePackageName) {
-    try {
-      boolean directory = classpathElement.isDirectory();
-
-      // Compute size and hash in one pass over file or directory.
-      Hasher hasher = Hashing.md5().newHasher();
-      OutputStream hashStream = Funnels.asOutputStream(hasher);
-      CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream);
-
-      if (!directory) {
-        // Files are staged as-is.
-        Files.asByteSource(classpathElement).copyTo(countingOutputStream);
-      } else {
-        // Directories are recursively zipped.
-        ZipFiles.zipDirectory(classpathElement, countingOutputStream);
-      }
-
-      long size = countingOutputStream.getCount();
-      String hash = Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes());
-
-      // Create the DataflowPackage with staging name and location.
-      String uniqueName = getUniqueContentName(classpathElement, hash);
-      String resourcePath = IOChannelUtils.resolve(stagingPath, uniqueName);
-      DataflowPackage target = new DataflowPackage();
-      target.setName(overridePackageName != null ? overridePackageName : uniqueName);
-      target.setLocation(resourcePath);
-
-      return new PackageAttributes(size, hash, directory, target);
-    } catch (IOException e) {
-      throw new RuntimeException("Package setup failure for " + classpathElement, e);
-    }
-  }
-
-  /**
-   * Transfers the classpath elements to the staging location.
-   *
-   * @param classpathElements The elements to stage.
-   * @param stagingPath The base location to stage the elements to.
-   * @return A list of cloud workflow packages, each representing a classpath element.
-   */
-  public static List<DataflowPackage> stageClasspathElements(
-      Collection<String> classpathElements, String stagingPath) {
-    return stageClasspathElements(classpathElements, stagingPath, Sleeper.DEFAULT);
-  }
-
-  // Visible for testing.
-  static List<DataflowPackage> stageClasspathElements(
-      Collection<String> classpathElements, String stagingPath,
-      Sleeper retrySleeper) {
-    LOG.info("Uploading {} files from PipelineOptions.filesToStage to staging location to "
-        + "prepare for execution.", classpathElements.size());
-
-    if (classpathElements.size() > SANE_CLASSPATH_SIZE) {
-      LOG.warn("Your classpath contains {} elements, which Google Cloud Dataflow automatically "
-          + "copies to all workers. Having this many entries on your classpath may be indicative "
-          + "of an issue in your pipeline. You may want to consider trimming the classpath to "
-          + "necessary dependencies only, using --filesToStage pipeline option to override "
-          + "what files are being staged, or bundling several dependencies into one.",
-          classpathElements.size());
-    }
-
-    ArrayList<DataflowPackage> packages = new ArrayList<>();
-
-    if (stagingPath == null) {
-      throw new IllegalArgumentException(
-          "Can't stage classpath elements on because no staging location has been provided");
-    }
-
-    int numUploaded = 0;
-    int numCached = 0;
-    for (String classpathElement : classpathElements) {
-      String packageName = null;
-      if (classpathElement.contains("=")) {
-        String[] components = classpathElement.split("=", 2);
-        packageName = components[0];
-        classpathElement = components[1];
-      }
-
-      File file = new File(classpathElement);
-      if (!file.exists()) {
-        LOG.warn("Skipping non-existent classpath element {} that was specified.",
-            classpathElement);
-        continue;
-      }
-
-      PackageAttributes attributes = createPackageAttributes(file, stagingPath, packageName);
-
-      DataflowPackage workflowPackage = attributes.getDataflowPackage();
-      packages.add(workflowPackage);
-      String target = workflowPackage.getLocation();
-
-      // TODO: Should we attempt to detect the Mime type rather than
-      // always using MimeTypes.BINARY?
-      try {
-        try {
-          long remoteLength = IOChannelUtils.getSizeBytes(target);
-          if (remoteLength == attributes.getSize()) {
-            LOG.debug("Skipping classpath element already staged: {} at {}",
-                classpathElement, target);
-            numCached++;
-            continue;
-          }
-        } catch (FileNotFoundException expected) {
-          // If the file doesn't exist, it means we need to upload it.
-        }
-
-        // Upload file, retrying on failure.
-        AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
-            MAX_ATTEMPTS,
-            INITIAL_BACKOFF_INTERVAL_MS);
-        while (true) {
-          try {
-            LOG.debug("Uploading classpath element {} to {}", classpathElement, target);
-            try (WritableByteChannel writer = IOChannelUtils.create(target, MimeTypes.BINARY)) {
-              copyContent(classpathElement, writer);
-            }
-            numUploaded++;
-            break;
-          } catch (IOException e) {
-            if (ERROR_EXTRACTOR.accessDenied(e)) {
-              String errorMessage = String.format(
-                  "Uploaded failed due to permissions error, will NOT retry staging "
-                  + "of classpath %s. Please verify credentials are valid and that you have "
-                  + "write access to %s. Stale credentials can be resolved by executing "
-                  + "'gcloud auth login'.", classpathElement, target);
-              LOG.error(errorMessage);
-              throw new IOException(errorMessage, e);
-            } else if (!backoff.atMaxAttempts()) {
-              LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
-                  classpathElement, e);
-              BackOffUtils.next(retrySleeper, backoff);
-            } else {
-              // Rethrow last error, to be included as a cause in the catch below.
-              LOG.error("Upload failed, will NOT retry staging of classpath: {}",
-                  classpathElement, e);
-              throw e;
-            }
-          }
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Could not stage classpath element: " + classpathElement, e);
-      }
-    }
-
-    LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, "
-        + "{} files cached",
-        numUploaded, numCached);
-
-    return packages;
-  }
-
-  /**
-   * Returns a unique name for a file with a given content hash.
-   *
-   * <p>Directory paths are removed. Example:
-   * <pre>
-   * dir="a/b/c/d", contentHash="f000" => d-f000.jar
-   * file="a/b/c/d.txt", contentHash="f000" => d-f000.txt
-   * file="a/b/c/d", contentHash="f000" => d-f000
-   * </pre>
-   */
-  static String getUniqueContentName(File classpathElement, String contentHash) {
-    String fileName = Files.getNameWithoutExtension(classpathElement.getAbsolutePath());
-    String fileExtension = Files.getFileExtension(classpathElement.getAbsolutePath());
-    if (classpathElement.isDirectory()) {
-      return fileName + "-" + contentHash + ".jar";
-    } else if (fileExtension.isEmpty()) {
-      return fileName + "-" + contentHash;
-    }
-    return fileName + "-" + contentHash + "." + fileExtension;
-  }
-
-  /**
-   * Copies the contents of the classpathElement to the output channel.
-   *
-   * <p>If the classpathElement is a directory, a Zip stream is constructed on the fly,
-   * otherwise the file contents are copied as-is.
-   *
-   * <p>The output channel is not closed.
-   */
-  private static void copyContent(String classpathElement, WritableByteChannel outputChannel)
-      throws IOException {
-    final File classpathElementFile = new File(classpathElement);
-    if (classpathElementFile.isDirectory()) {
-      ZipFiles.zipDirectory(classpathElementFile, Channels.newOutputStream(outputChannel));
-    } else {
-      Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel));
-    }
-  }
-  /**
-   * Holds the metadata necessary to stage a file or confirm that a staged file has not changed.
-   */
-  static class PackageAttributes {
-    private final boolean directory;
-    private final long size;
-    private final String hash;
-    private DataflowPackage dataflowPackage;
-
-    public PackageAttributes(long size, String hash, boolean directory,
-        DataflowPackage dataflowPackage) {
-      this.size = size;
-      this.hash = Objects.requireNonNull(hash, "hash");
-      this.directory = directory;
-      this.dataflowPackage = Objects.requireNonNull(dataflowPackage, "dataflowPackage");
-    }
-
-    /**
-     * @return the dataflowPackage
-     */
-    public DataflowPackage getDataflowPackage() {
-      return dataflowPackage;
-    }
-
-    /**
-     * @return the directory
-     */
-    public boolean isDirectory() {
-      return directory;
-    }
-
-    /**
-     * @return the size
-     */
-    public long getSize() {
-      return size;
-    }
-
-    /**
-     * @return the hash
-     */
-    public String getHash() {
-      return hash;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java
deleted file mode 100644
index f6c6a71..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java
+++ /dev/null
@@ -1,30 +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 com.google.cloud.dataflow.sdk.util;
-
-import com.google.api.services.dataflow.model.DataflowPackage;
-
-import java.util.List;
-
-/**
- * Interface for staging files needed for running a Dataflow pipeline.
- */
-public interface Stager {
-  /* Stage files and return a list of packages. */
-  public List<DataflowPackage> stageFiles();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
new file mode 100644
index 0000000..6bbafdd
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
@@ -0,0 +1,50 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.io.PrintStream;
+
+/**
+ * Options that are used to configure the {@link BlockingDataflowPipelineRunner}.
+ */
+@Description("Configure options on the BlockingDataflowPipelineRunner.")
+public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions {
+  /**
+   * Output stream for job status messages.
+   */
+  @Description("Where messages generated during execution of the Dataflow job will be output.")
+  @JsonIgnore
+  @Hidden
+  @Default.InstanceFactory(StandardOutputFactory.class)
+  PrintStream getJobMessageOutput();
+  void setJobMessageOutput(PrintStream value);
+
+  /**
+   * Returns a default of {@link System#out}.
+   */
+  public static class StandardOutputFactory implements DefaultValueFactory<PrintStream> {
+    @Override
+    public PrintStream create(PipelineOptions options) {
+      return System.out;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
new file mode 100644
index 0000000..3f0503e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.services.clouddebugger.v2.model.Debuggee;
+
+import javax.annotation.Nullable;
+
+/**
+ * Options for controlling Cloud Debugger.
+ */
+@Description("[Experimental] Used to configure the Cloud Debugger")
+@Experimental
+@Hidden
+public interface CloudDebuggerOptions {
+
+  /** Whether to enable the Cloud Debugger snapshot agent for the current job. */
+  @Description("Whether to enable the Cloud Debugger snapshot agent for the current job.")
+  boolean getEnableCloudDebugger();
+  void setEnableCloudDebugger(boolean enabled);
+
+  /** The Cloud Debugger debuggee to associate with. This should not be set directly. */
+  @Description("The Cloud Debugger debuggee to associate with. This should not be set directly.")
+  @Hidden
+  @Nullable Debuggee getDebuggee();
+  void setDebuggee(Debuggee debuggee);
+
+  /** The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. */
+  @Description(
+      "The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. "
+      + "Should be a double between 0 and 1. "
+      + "Snapshots will be cancelled if evaluating conditions takes more than this ratio of time.")
+  @Default.Double(0.01)
+  double getMaxConditionCost();
+  void setMaxConditionCost(double maxConditionCost);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
new file mode 100644
index 0000000..1be93eb
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
@@ -0,0 +1,254 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.cloud.dataflow.sdk.util.DataflowPathValidator;
+import com.google.cloud.dataflow.sdk.util.DataflowTransport;
+import com.google.cloud.dataflow.sdk.util.GcsStager;
+import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
+import com.google.cloud.dataflow.sdk.util.PathValidator;
+import com.google.cloud.dataflow.sdk.util.Stager;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Internal. Options used to control execution of the Dataflow SDK for
+ * debugging and testing purposes.
+ */
+@Description("[Internal] Options used to control execution of the Dataflow SDK for "
+    + "debugging and testing purposes.")
+@Hidden
+public interface DataflowPipelineDebugOptions extends PipelineOptions {
+
+  /**
+   * The list of backend experiments to enable.
+   *
+   * <p>Dataflow provides a number of experimental features that can be enabled
+   * with this flag.
+   *
+   * <p>Please sync with the Dataflow team before enabling any experiments.
+   */
+  @Description("[Experimental] Dataflow provides a number of experimental features that can "
+      + "be enabled with this flag. Please sync with the Dataflow team before enabling any "
+      + "experiments.")
+  @Experimental
+  List<String> getExperiments();
+  void setExperiments(List<String> value);
+
+  /**
+   * The root URL for the Dataflow API. {@code dataflowEndpoint} can override this value
+   * if it contains an absolute URL, otherwise {@code apiRootUrl} will be combined with
+   * {@code dataflowEndpoint} to generate the full URL to communicate with the Dataflow API.
+   */
+  @Description("The root URL for the Dataflow API. dataflowEndpoint can override this "
+      + "value if it contains an absolute URL, otherwise apiRootUrl will be combined with "
+      + "dataflowEndpoint to generate the full URL to communicate with the Dataflow API.")
+  @Default.String(Dataflow.DEFAULT_ROOT_URL)
+  String getApiRootUrl();
+  void setApiRootUrl(String value);
+
+  /**
+   * Dataflow endpoint to use.
+   *
+   * <p>Defaults to the current version of the Google Cloud Dataflow
+   * API, at the time the current SDK version was released.
+   *
+   * <p>If the string contains "://", then this is treated as a URL,
+   * otherwise {@link #getApiRootUrl()} is used as the root
+   * URL.
+   */
+  @Description("The URL for the Dataflow API. If the string contains \"://\", this"
+      + " will be treated as the entire URL, otherwise will be treated relative to apiRootUrl.")
+  @Default.String(Dataflow.DEFAULT_SERVICE_PATH)
+  String getDataflowEndpoint();
+  void setDataflowEndpoint(String value);
+
+  /**
+   * The path to write the translated Dataflow job specification out to
+   * at job submission time. The Dataflow job specification will be represented in JSON
+   * format.
+   */
+  @Description("The path to write the translated Dataflow job specification out to "
+      + "at job submission time. The Dataflow job specification will be represented in JSON "
+      + "format.")
+  String getDataflowJobFile();
+  void setDataflowJobFile(String value);
+
+  /**
+   * The class of the validator that should be created and used to validate paths.
+   * If pathValidator has not been set explicitly, an instance of this class will be
+   * constructed and used as the path validator.
+   */
+  @Description("The class of the validator that should be created and used to validate paths. "
+      + "If pathValidator has not been set explicitly, an instance of this class will be "
+      + "constructed and used as the path validator.")
+  @Default.Class(DataflowPathValidator.class)
+  Class<? extends PathValidator> getPathValidatorClass();
+  void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
+
+  /**
+   * The path validator instance that should be used to validate paths.
+   * If no path validator has been set explicitly, the default is to use the instance factory that
+   * constructs a path validator based upon the currently set pathValidatorClass.
+   */
+  @JsonIgnore
+  @Description("The path validator instance that should be used to validate paths. "
+      + "If no path validator has been set explicitly, the default is to use the instance factory "
+      + "that constructs a path validator based upon the currently set pathValidatorClass.")
+  @Default.InstanceFactory(PathValidatorFactory.class)
+  PathValidator getPathValidator();
+  void setPathValidator(PathValidator validator);
+
+  /**
+   * The class responsible for staging resources to be accessible by workers
+   * during job execution. If stager has not been set explicitly, an instance of this class
+   * will be created and used as the resource stager.
+   */
+  @Description("The class of the stager that should be created and used to stage resources. "
+      + "If stager has not been set explicitly, an instance of the this class will be created "
+      + "and used as the resource stager.")
+  @Default.Class(GcsStager.class)
+  Class<? extends Stager> getStagerClass();
+  void setStagerClass(Class<? extends Stager> stagerClass);
+
+  /**
+   * The resource stager instance that should be used to stage resources.
+   * If no stager has been set explicitly, the default is to use the instance factory
+   * that constructs a resource stager based upon the currently set stagerClass.
+   */
+  @JsonIgnore
+  @Description("The resource stager instance that should be used to stage resources. "
+      + "If no stager has been set explicitly, the default is to use the instance factory "
+      + "that constructs a resource stager based upon the currently set stagerClass.")
+  @Default.InstanceFactory(StagerFactory.class)
+  Stager getStager();
+  void setStager(Stager stager);
+
+  /**
+   * An instance of the Dataflow client. Defaults to creating a Dataflow client
+   * using the current set of options.
+   */
+  @JsonIgnore
+  @Description("An instance of the Dataflow client. Defaults to creating a Dataflow client "
+      + "using the current set of options.")
+  @Default.InstanceFactory(DataflowClientFactory.class)
+  Dataflow getDataflowClient();
+  void setDataflowClient(Dataflow value);
+
+  /** Returns the default Dataflow client built from the passed in PipelineOptions. */
+  public static class DataflowClientFactory implements DefaultValueFactory<Dataflow> {
+    @Override
+    public Dataflow create(PipelineOptions options) {
+        return DataflowTransport.newDataflowClient(
+            options.as(DataflowPipelineOptions.class)).build();
+    }
+  }
+
+  /**
+   * Whether to update the currently running pipeline with the same name as this one.
+   *
+   * @deprecated This property is replaced by {@link DataflowPipelineOptions#getUpdate()}
+   */
+  @Deprecated
+  @Description("If set, replace the existing pipeline with the name specified by --jobName with "
+      + "this pipeline, preserving state.")
+  boolean getUpdate();
+  @Deprecated
+  void setUpdate(boolean value);
+
+  /**
+   * Mapping of old PTranform names to new ones, specified as JSON
+   * <code>{"oldName":"newName",...}</code>. To mark a transform as deleted, make newName the
+   * empty string.
+   */
+  @JsonIgnore
+  @Description(
+      "Mapping of old PTranform names to new ones, specified as JSON "
+      + "{\"oldName\":\"newName\",...}. To mark a transform as deleted, make newName the empty "
+      + "string.")
+  Map<String, String> getTransformNameMapping();
+  void setTransformNameMapping(Map<String, String> value);
+
+  /**
+   * Custom windmill_main binary to use with the streaming runner.
+   */
+  @Description("Custom windmill_main binary to use with the streaming runner")
+  String getOverrideWindmillBinary();
+  void setOverrideWindmillBinary(String value);
+
+  /**
+   * Number of threads to use on the Dataflow worker harness. If left unspecified,
+   * the Dataflow service will compute an appropriate number of threads to use.
+   */
+  @Description("Number of threads to use on the Dataflow worker harness. If left unspecified, "
+      + "the Dataflow service will compute an appropriate number of threads to use.")
+  int getNumberOfWorkerHarnessThreads();
+  void setNumberOfWorkerHarnessThreads(int value);
+
+  /**
+   * If {@literal true}, save a heap dump before killing a thread or process which is GC
+   * thrashing or out of memory. The location of the heap file will either be echoed back
+   * to the user, or the user will be given the opportunity to download the heap file.
+   *
+   * <p>
+   * CAUTION: Heap dumps can of comparable size to the default boot disk. Consider increasing
+   * the boot disk size before setting this flag to true.
+   */
+  @Description("If {@literal true}, save a heap dump before killing a thread or process "
+      + "which is GC thrashing or out of memory.")
+  boolean getDumpHeapOnOOM();
+  void setDumpHeapOnOOM(boolean dumpHeapBeforeExit);
+
+  /**
+   * Creates a {@link PathValidator} object using the class specified in
+   * {@link #getPathValidatorClass()}.
+   */
+  public static class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
+      @Override
+      public PathValidator create(PipelineOptions options) {
+      DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
+      return InstanceBuilder.ofType(PathValidator.class)
+          .fromClass(debugOptions.getPathValidatorClass())
+          .fromFactoryMethod("fromOptions")
+          .withArg(PipelineOptions.class, options)
+          .build();
+    }
+  }
+
+  /**
+   * Creates a {@link Stager} object using the class specified in
+   * {@link #getStagerClass()}.
+   */
+  public static class StagerFactory implements DefaultValueFactory<Stager> {
+      @Override
+      public Stager create(PipelineOptions options) {
+      DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
+      return InstanceBuilder.ofType(Stager.class)
+          .fromClass(debugOptions.getStagerClass())
+          .fromFactoryMethod("fromOptions")
+          .withArg(PipelineOptions.class, options)
+          .build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
new file mode 100644
index 0000000..dbfafd1
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
@@ -0,0 +1,115 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import com.google.cloud.dataflow.sdk.runners.DataflowPipeline;
+import com.google.common.base.MoreObjects;
+
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+/**
+ * Options that can be used to configure the {@link DataflowPipeline}.
+ */
+@Description("Options that configure the Dataflow pipeline.")
+public interface DataflowPipelineOptions extends
+    PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions,
+    DataflowPipelineWorkerPoolOptions, BigQueryOptions,
+    GcsOptions, StreamingOptions, CloudDebuggerOptions, DataflowWorkerLoggingOptions,
+    DataflowProfilingOptions, PubsubOptions {
+
+  @Description("Project id. Required when running a Dataflow in the cloud. "
+      + "See https://cloud.google.com/storage/docs/projects for further details.")
+  @Override
+  @Validation.Required
+  @Default.InstanceFactory(DefaultProjectFactory.class)
+  String getProject();
+  @Override
+  void setProject(String value);
+
+  /**
+   * GCS path for staging local files, e.g. gs://bucket/object
+   *
+   * <p>Must be a valid Cloud Storage URL, beginning with the prefix "gs://"
+   *
+   * <p>At least one of {@link PipelineOptions#getTempLocation()} or {@link #getStagingLocation()}
+   * must be set. If {@link #getStagingLocation()} is not set, then the Dataflow
+   * pipeline defaults to using {@link PipelineOptions#getTempLocation()}.
+   */
+  @Description("GCS path for staging local files, e.g. \"gs://bucket/object\". "
+      + "Must be a valid Cloud Storage URL, beginning with the prefix \"gs://\". "
+      + "At least one of stagingLocation or tempLocation must be set. If stagingLocation is unset, "
+      + "defaults to using tempLocation.")
+  String getStagingLocation();
+  void setStagingLocation(String value);
+
+  /**
+   * The Dataflow job name is used as an idempotence key within the Dataflow service.
+   * If there is an existing job that is currently active, another active job with the same
+   * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.
+   */
+  @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. "
+      + "If there is an existing job that is currently active, another active job with the same "
+      + "name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.")
+  @Default.InstanceFactory(JobNameFactory.class)
+  String getJobName();
+  void setJobName(String value);
+
+  /**
+   * Whether to update the currently running pipeline with the same name as this one.
+   */
+  @Override
+  @SuppressWarnings("deprecation") // base class member deprecated in favor of this one.
+  @Description(
+      "If set, replace the existing pipeline with the name specified by --jobName with "
+          + "this pipeline, preserving state.")
+  boolean getUpdate();
+  @Override
+  @SuppressWarnings("deprecation") // base class member deprecated in favor of this one.
+  void setUpdate(boolean value);
+
+  /**
+   * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the
+   * local system user name (if available), and the current time. The normalization makes sure that
+   * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40
+   * characters.
+   *
+   * <p>This job name factory is only able to generate one unique name per second per application
+   * and user combination.
+   */
+  public static class JobNameFactory implements DefaultValueFactory<String> {
+    private static final DateTimeFormatter FORMATTER =
+        DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC);
+
+    @Override
+    public String create(PipelineOptions options) {
+      String appName = options.as(ApplicationNameOptions.class).getAppName();
+      String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow"
+          : appName.toLowerCase()
+                   .replaceAll("[^a-z0-9]", "0")
+                   .replaceAll("^[^a-z]", "a");
+      String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), "");
+      String normalizedUserName = userName.toLowerCase()
+                                          .replaceAll("[^a-z0-9]", "0");
+      String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis());
+      return normalizedAppName + "-" + normalizedUserName + "-" + datePart;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
new file mode 100644
index 0000000..44a9b00
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
@@ -0,0 +1,258 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.List;
+
+/**
+ * Options that are used to configure the Dataflow pipeline worker pool.
+ */
+@Description("Options that are used to configure the Dataflow pipeline worker pool.")
+public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions {
+  /**
+   * Number of workers to use when executing the Dataflow job. Note that selection of an autoscaling
+   * algorithm other then {@code NONE} will affect the size of the worker pool. If left unspecified,
+   * the Dataflow service will determine the number of workers.
+   */
+  @Description("Number of workers to use when executing the Dataflow job. Note that "
+      + "selection of an autoscaling algorithm other then \"NONE\" will affect the "
+      + "size of the worker pool. If left unspecified, the Dataflow service will "
+      + "determine the number of workers.")
+  int getNumWorkers();
+  void setNumWorkers(int value);
+
+  /**
+   * Type of autoscaling algorithm to use.
+   */
+  @Experimental(Experimental.Kind.AUTOSCALING)
+  public enum AutoscalingAlgorithmType {
+    /** Use numWorkers machines. Do not autoscale the worker pool. */
+    NONE("AUTOSCALING_ALGORITHM_NONE"),
+
+    @Deprecated
+    BASIC("AUTOSCALING_ALGORITHM_BASIC"),
+
+    /** Autoscale the workerpool based on throughput (up to maxNumWorkers). */
+    THROUGHPUT_BASED("AUTOSCALING_ALGORITHM_BASIC");
+
+    private final String algorithm;
+
+    private AutoscalingAlgorithmType(String algorithm) {
+      this.algorithm = algorithm;
+    }
+
+    /** Returns the string representation of this type. */
+    public String getAlgorithm() {
+      return this.algorithm;
+    }
+  }
+
+  /**
+   * [Experimental] The autoscaling algorithm to use for the workerpool.
+   *
+   * <ul>
+   *   <li>NONE: does not change the size of the worker pool.</li>
+   *   <li>BASIC: autoscale the worker pool size up to maxNumWorkers until the job completes.</li>
+   *   <li>THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).
+   *   </li>
+   * </ul>
+   */
+  @Description("[Experimental] The autoscaling algorithm to use for the workerpool. "
+      + "NONE: does not change the size of the worker pool. "
+      + "BASIC (deprecated): autoscale the worker pool size up to maxNumWorkers until the job "
+      + "completes. "
+      + "THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).")
+  @Experimental(Experimental.Kind.AUTOSCALING)
+  AutoscalingAlgorithmType getAutoscalingAlgorithm();
+  void setAutoscalingAlgorithm(AutoscalingAlgorithmType value);
+
+  /**
+   * The maximum number of workers to use for the workerpool. This options limits the size of the
+   * workerpool for the lifetime of the job, including
+   * <a href="https://cloud.google.com/dataflow/pipelines/updating-a-pipeline">pipeline updates</a>.
+   * If left unspecified, the Dataflow service will compute a ceiling.
+   */
+  @Description("The maximum number of workers to use for the workerpool. This options limits the "
+      + "size of the workerpool for the lifetime of the job, including pipeline updates. "
+      + "If left unspecified, the Dataflow service will compute a ceiling.")
+  int getMaxNumWorkers();
+  void setMaxNumWorkers(int value);
+
+  /**
+   * Remote worker disk size, in gigabytes, or 0 to use the default size.
+   */
+  @Description("Remote worker disk size, in gigabytes, or 0 to use the default size.")
+  int getDiskSizeGb();
+  void setDiskSizeGb(int value);
+
+  /**
+   * Docker container image that executes Dataflow worker harness, residing in Google Container
+   * Registry.
+   */
+  @Default.InstanceFactory(WorkerHarnessContainerImageFactory.class)
+  @Description("Docker container image that executes Dataflow worker harness, residing in Google "
+      + " Container Registry.")
+  @Hidden
+  String getWorkerHarnessContainerImage();
+  void setWorkerHarnessContainerImage(String value);
+
+  /**
+   * Returns the default Docker container image that executes Dataflow worker harness, residing in
+   * Google Container Registry.
+   */
+  public static class WorkerHarnessContainerImageFactory
+      implements DefaultValueFactory<String> {
+    @Override
+    public String create(PipelineOptions options) {
+      DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+      if (dataflowOptions.isStreaming()) {
+        return DataflowPipelineRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
+      } else {
+        return DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
+      }
+    }
+  }
+
+  /**
+   * GCE <a href="https://cloud.google.com/compute/docs/networking">network</a> for launching
+   * workers.
+   *
+   * <p>Default is up to the Dataflow service.
+   */
+  @Description("GCE network for launching workers. For more information, see the reference "
+      + "documentation https://cloud.google.com/compute/docs/networking. "
+      + "Default is up to the Dataflow service.")
+  String getNetwork();
+  void setNetwork(String value);
+
+  /**
+   * GCE <a href="https://cloud.google.com/compute/docs/networking">subnetwork</a> for launching
+   * workers.
+   *
+   * <p>Default is up to the Dataflow service. Expected format is
+   * regions/REGION/subnetworks/SUBNETWORK.
+   *
+   * <p>You may also need to specify network option.
+   */
+  @Description("GCE subnetwork for launching workers. For more information, see the reference "
+      + "documentation https://cloud.google.com/compute/docs/networking. "
+      + "Default is up to the Dataflow service.")
+  String getSubnetwork();
+  void setSubnetwork(String value);
+
+  /**
+   * GCE <a href="https://developers.google.com/compute/docs/zones"
+   * >availability zone</a> for launching workers.
+   *
+   * <p>Default is up to the Dataflow service.
+   */
+  @Description("GCE availability zone for launching workers. See "
+      + "https://developers.google.com/compute/docs/zones for a list of valid options. "
+      + "Default is up to the Dataflow service.")
+  String getZone();
+  void setZone(String value);
+
+  /**
+   * Machine type to create Dataflow worker VMs as.
+   *
+   * <p>See <a href="https://cloud.google.com/compute/docs/machine-types">GCE machine types</a>
+   * for a list of valid options.
+   *
+   * <p>If unset, the Dataflow service will choose a reasonable default.
+   */
+  @Description("Machine type to create Dataflow worker VMs as. See "
+      + "https://cloud.google.com/compute/docs/machine-types for a list of valid options. "
+      + "If unset, the Dataflow service will choose a reasonable default.")
+  String getWorkerMachineType();
+  void setWorkerMachineType(String value);
+
+  /**
+   * The policy for tearing down the workers spun up by the service.
+   */
+  public enum TeardownPolicy {
+    /**
+     * All VMs created for a Dataflow job are deleted when the job finishes, regardless of whether
+     * it fails or succeeds.
+     */
+    TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"),
+    /**
+     * All VMs created for a Dataflow job are left running when the job finishes, regardless of
+     * whether it fails or succeeds.
+     */
+    TEARDOWN_NEVER("TEARDOWN_NEVER"),
+    /**
+     * All VMs created for a Dataflow job are deleted when the job succeeds, but are left running
+     * when it fails. (This is typically used for debugging failing jobs by SSHing into the
+     * workers.)
+     */
+    TEARDOWN_ON_SUCCESS("TEARDOWN_ON_SUCCESS");
+
+    private final String teardownPolicy;
+
+    private TeardownPolicy(String teardownPolicy) {
+      this.teardownPolicy = teardownPolicy;
+    }
+
+    public String getTeardownPolicyName() {
+      return this.teardownPolicy;
+    }
+  }
+
+  /**
+   * The teardown policy for the VMs.
+   *
+   * <p>If unset, the Dataflow service will choose a reasonable default.
+   */
+  @Description("The teardown policy for the VMs. If unset, the Dataflow service will "
+      + "choose a reasonable default.")
+  TeardownPolicy getTeardownPolicy();
+  void setTeardownPolicy(TeardownPolicy value);
+
+  /**
+   * List of local files to make available to workers.
+   *
+   * <p>Files are placed on the worker's classpath.
+   *
+   * <p>The default value is the list of jars from the main program's classpath.
+   */
+  @Description("Files to stage on GCS and make available to workers. "
+      + "Files are placed on the worker's classpath. "
+      + "The default value is all files from the classpath.")
+  @JsonIgnore
+  List<String> getFilesToStage();
+  void setFilesToStage(List<String> value);
+
+  /**
+   * Specifies what type of persistent disk should be used. The value should be a full or partial
+   * URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For
+   * more information, see the
+   * <a href="https://cloud.google.com/compute/docs/reference/latest/diskTypes">API reference
+   * documentation for DiskTypes</a>.
+   */
+  @Description("Specifies what type of persistent disk should be used. The value should be a full "
+      + "or partial URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For "
+      + "more information, see the API reference documentation for DiskTypes: "
+      + "https://cloud.google.com/compute/docs/reference/latest/diskTypes")
+  String getWorkerDiskType();
+  void setWorkerDiskType(String value);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
new file mode 100644
index 0000000..3cd7b03
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
@@ -0,0 +1,48 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import java.util.HashMap;
+
+/**
+ * Options for controlling profiling of pipeline execution.
+ */
+@Description("[Experimental] Used to configure profiling of the Dataflow pipeline")
+@Experimental
+@Hidden
+public interface DataflowProfilingOptions {
+
+  @Description("Whether to periodically dump profiling information to local disk.\n"
+      + "WARNING: Enabling this option may fill local disk with profiling information.")
+  boolean getEnableProfilingAgent();
+  void setEnableProfilingAgent(boolean enabled);
+
+  @Description(
+      "[INTERNAL] Additional configuration for the profiling agent. Not typically necessary.")
+  @Hidden
+  DataflowProfilingAgentConfiguration getProfilingAgentConfiguration();
+  void setProfilingAgentConfiguration(DataflowProfilingAgentConfiguration configuration);
+
+  /**
+   * Configuration the for profiling agent.
+   */
+  public static class DataflowProfilingAgentConfiguration extends HashMap<String, Object> {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
new file mode 100644
index 0000000..7705b66
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
@@ -0,0 +1,51 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.options;
+
+/**
+ * Options that are used exclusively within the Dataflow worker harness.
+ * These options have no effect at pipeline creation time.
+ */
+@Description("[Internal] Options that are used exclusively within the Dataflow worker harness. "
+    + "These options have no effect at pipeline creation time.")
+@Hidden
+public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions {
+  /**
+   * The identity of the worker running this pipeline.
+   */
+  @Description("The identity of the worker running this pipeline.")
+  String getWorkerId();
+  void setWorkerId(String value);
+
+  /**
+   * The identity of the Dataflow job.
+   */
+  @Description("The identity of the Dataflow job.")
+  String getJobId();
+  void setJobId(String value);
+
+  /**
+   * The size of the worker's in-memory cache, in megabytes.
+   *
+   * <p>Currently, this cache is used for storing read values of side inputs.
+   */
+  @Description("The size of the worker's in-memory cache, in megabytes.")
+  @Default.Integer(100)
+  Integer getWorkerCacheMb();
+  void setWorkerCacheMb(Integer value);
+}


[35/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java
deleted file mode 100644
index 8f8c653..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java
+++ /dev/null
@@ -1,39 +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 com.google.cloud.dataflow.sdk.runners;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.dataflow.model.Environment;
-
-/**
- * An instance of this class can be passed to the
- * {@link DataflowPipelineRunner} to add user defined hooks to be
- * invoked at various times during pipeline execution.
- */
-@Experimental
-public class DataflowPipelineRunnerHooks {
-  /**
-   * Allows the user to modify the environment of their job before their job is submitted
-   * to the service for execution.
-   *
-   * @param environment The environment of the job. Users can make change to this instance in order
-   *     to change the environment with which their job executes on the service.
-   */
-  public void modifyEnvironmentBeforeSubmission(Environment environment) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java
deleted file mode 100644
index fa7067e..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java
+++ /dev/null
@@ -1,1100 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString;
-import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray;
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.addList;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-import static com.google.cloud.dataflow.sdk.util.Structs.addObject;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.getString;
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.services.dataflow.model.AutoscalingSettings;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Disk;
-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.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.coders.IterableCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
-import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator;
-import com.google.cloud.dataflow.sdk.runners.dataflow.ReadTranslator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
-import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.util.AppliedCombineFn;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.DoFnInfo;
-import com.google.cloud.dataflow.sdk.util.OutputReference;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects
- * into Cloud Dataflow Service API {@link Job}s.
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class DataflowPipelineTranslator {
-  // Must be kept in sync with their internal counterparts.
-  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class);
-  private static final ObjectMapper MAPPER = new ObjectMapper();
-
-  /**
-   * A map from {@link PTransform} subclass to the corresponding
-   * {@link TransformTranslator} to use to translate that transform.
-   *
-   * <p>A static map that contains system-wide defaults.
-   */
-  private static Map<Class, TransformTranslator> transformTranslators =
-      new HashMap<>();
-
-  /** Provided configuration options. */
-  private final DataflowPipelineOptions options;
-
-  /**
-   * Constructs a translator from the provided options.
-   *
-   * @param options Properties that configure the translator.
-   *
-   * @return The newly created translator.
-   */
-  public static DataflowPipelineTranslator fromOptions(
-      DataflowPipelineOptions options) {
-    return new DataflowPipelineTranslator(options);
-  }
-
-  private DataflowPipelineTranslator(DataflowPipelineOptions options) {
-    this.options = options;
-  }
-
-  /**
-   * Translates a {@link Pipeline} into a {@code JobSpecification}.
-   */
-  public JobSpecification translate(
-      Pipeline pipeline,
-      DataflowPipelineRunner runner,
-      List<DataflowPackage> packages) {
-
-    Translator translator = new Translator(pipeline, runner);
-    Job result = translator.translate(packages);
-    return new JobSpecification(result, Collections.unmodifiableMap(translator.stepNames));
-  }
-
-  /**
-   * The result of a job translation.
-   *
-   * <p>Used to pass the result {@link Job} and any state that was used to construct the job that
-   * may be of use to other classes (eg the {@link PTransform} to StepName mapping).
-   */
-  public static class JobSpecification {
-    private final Job job;
-    private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
-
-    public JobSpecification(Job job, Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
-      this.job = job;
-      this.stepNames = stepNames;
-    }
-
-    public Job getJob() {
-      return job;
-    }
-
-    /**
-     * Returns the mapping of {@link AppliedPTransform AppliedPTransforms} to the internal step
-     * name for that {@code AppliedPTransform}.
-     */
-    public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
-      return stepNames;
-    }
-  }
-
-  /**
-   * Renders a {@link Job} as a string.
-   */
-  public static String jobToString(Job job) {
-    try {
-      return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(job);
-    } catch (JsonProcessingException exc) {
-      throw new IllegalStateException("Failed to render Job as String.", exc);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Records that instances of the specified PTransform class
-   * should be translated by default by the corresponding
-   * {@link TransformTranslator}.
-   */
-  public static <TransformT extends PTransform> void registerTransformTranslator(
-      Class<TransformT> transformClass,
-      TransformTranslator<? extends TransformT> transformTranslator) {
-    if (transformTranslators.put(transformClass, transformTranslator) != null) {
-      throw new IllegalArgumentException(
-          "defining multiple translators for " + transformClass);
-    }
-  }
-
-  /**
-   * Returns the {@link TransformTranslator} to use for instances of the
-   * specified PTransform class, or null if none registered.
-   */
-  public <TransformT extends PTransform>
-      TransformTranslator<TransformT> getTransformTranslator(Class<TransformT> transformClass) {
-    return transformTranslators.get(transformClass);
-  }
-
-  /**
-   * A {@link TransformTranslator} knows how to translate
-   * a particular subclass of {@link PTransform} for the
-   * Cloud Dataflow service. It does so by
-   * mutating the {@link TranslationContext}.
-   */
-  public interface TransformTranslator<TransformT extends PTransform> {
-    public void translate(TransformT transform,
-                          TranslationContext context);
-  }
-
-  /**
-   * The interface provided to registered callbacks for interacting
-   * with the {@link DataflowPipelineRunner}, including reading and writing the
-   * values of {@link PCollection}s and side inputs ({@link PCollectionView}s).
-   */
-  public interface TranslationContext {
-    /**
-     * Returns the configured pipeline options.
-     */
-    DataflowPipelineOptions getPipelineOptions();
-
-    /**
-     * Returns the input of the currently being translated transform.
-     */
-    <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
-
-    /**
-     * Returns the output of the currently being translated transform.
-     */
-    <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
-
-    /**
-     * Returns the full name of the currently being translated transform.
-     */
-    String getFullName(PTransform<?, ?> transform);
-
-    /**
-     * Adds a step to the Dataflow workflow for the given transform, with
-     * the given Dataflow step type.
-     * This step becomes "current" for the purpose of {@link #addInput} and
-     * {@link #addOutput}.
-     */
-    public void addStep(PTransform<?, ?> transform, String type);
-
-    /**
-     * Adds a pre-defined step to the Dataflow workflow. The given PTransform should be
-     * consistent with the Step, in terms of input, output and coder types.
-     *
-     * <p>This is a low-level operation, when using this method it is up to
-     * the caller to ensure that names do not collide.
-     */
-    public void addStep(PTransform<?, ? extends PValue> transform, Step step);
-
-    /**
-     * Sets the encoding for the current Dataflow step.
-     */
-    public void addEncodingInput(Coder<?> value);
-
-    /**
-     * Adds an input with the given name and value to the current
-     * Dataflow step.
-     */
-    public void addInput(String name, Boolean value);
-
-    /**
-     * Adds an input with the given name and value to the current
-     * Dataflow step.
-     */
-    public void addInput(String name, String value);
-
-    /**
-     * Adds an input with the given name and value to the current
-     * Dataflow step.
-     */
-    public void addInput(String name, Long value);
-
-    /**
-     * Adds an input with the given name to the previously added Dataflow
-     * step, coming from the specified input PValue.
-     */
-    public void addInput(String name, PInput value);
-
-    /**
-     * Adds an input that is a dictionary of strings to objects.
-     */
-    public void addInput(String name, Map<String, Object> elements);
-
-    /**
-     * Adds an input that is a list of objects.
-     */
-    public void addInput(String name, List<? extends Map<String, Object>> elements);
-
-    /**
-     * Adds an output with the given name to the previously added
-     * Dataflow step, producing the specified output {@code PValue},
-     * including its {@code Coder} if a {@code TypedPValue}.  If the
-     * {@code PValue} is a {@code PCollection}, wraps its coder inside
-     * a {@code WindowedValueCoder}.
-     */
-    public void addOutput(String name, PValue value);
-
-    /**
-     * Adds an output with the given name to the previously added
-     * Dataflow step, producing the specified output {@code PValue},
-     * including its {@code Coder} if a {@code TypedPValue}.  If the
-     * {@code PValue} is a {@code PCollection}, wraps its coder inside
-     * a {@code ValueOnlyCoder}.
-     */
-    public void addValueOnlyOutput(String name, PValue value);
-
-    /**
-     * Adds an output with the given name to the previously added
-     * CollectionToSingleton Dataflow step, consuming the specified
-     * input {@code PValue} and producing the specified output
-     * {@code PValue}.  This step requires special treatment for its
-     * output encoding.
-     */
-    public void addCollectionToSingletonOutput(String name,
-                                               PValue inputValue,
-                                               PValue outputValue);
-
-    /**
-     * Encode a PValue reference as an output reference.
-     */
-    public OutputReference asOutputReference(PValue value);
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Translates a Pipeline into the Dataflow representation.
-   */
-  class Translator implements PipelineVisitor, TranslationContext {
-    /** The Pipeline to translate. */
-    private final Pipeline pipeline;
-
-    /** The runner which will execute the pipeline. */
-    private final DataflowPipelineRunner runner;
-
-    /** The Cloud Dataflow Job representation. */
-    private final Job job = new Job();
-
-    /**
-     * Translator is stateful, as addProperty calls refer to the current step.
-     */
-    private Step currentStep;
-
-    /**
-     * A Map from AppliedPTransform to their unique Dataflow step names.
-     */
-    private final Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
-
-    /**
-     * A Map from PValues to their output names used by their producer
-     * Dataflow steps.
-     */
-    private final Map<POutput, String> outputNames = new HashMap<>();
-
-    /**
-     * A Map from PValues to the Coders used for them.
-     */
-    private final Map<POutput, Coder<?>> outputCoders = new HashMap<>();
-
-    /**
-     * The transform currently being applied.
-     */
-    private AppliedPTransform<?, ?, ?> currentTransform;
-
-    /**
-     * Constructs a Translator that will translate the specified
-     * Pipeline into Dataflow objects.
-     */
-    public Translator(Pipeline pipeline, DataflowPipelineRunner runner) {
-      this.pipeline = pipeline;
-      this.runner = runner;
-    }
-
-    /**
-     * Translates this Translator's pipeline onto its writer.
-     * @return a Job definition filled in with the type of job, the environment,
-     * and the job steps.
-     */
-    public Job translate(List<DataflowPackage> packages) {
-      job.setName(options.getJobName().toLowerCase());
-
-      Environment environment = new Environment();
-      job.setEnvironment(environment);
-
-      try {
-        environment.setSdkPipelineOptions(
-            MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class));
-      } catch (IOException e) {
-        throw new IllegalArgumentException(
-            "PipelineOptions specified failed to serialize to JSON.", e);
-      }
-
-      WorkerPool workerPool = new WorkerPool();
-
-      if (options.getTeardownPolicy() != null) {
-        workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName());
-      }
-
-      if (options.isStreaming()) {
-        job.setType("JOB_TYPE_STREAMING");
-      } else {
-        job.setType("JOB_TYPE_BATCH");
-        workerPool.setDiskType(options.getWorkerDiskType());
-      }
-
-      if (options.getWorkerMachineType() != null) {
-        workerPool.setMachineType(options.getWorkerMachineType());
-      }
-
-      workerPool.setPackages(packages);
-      workerPool.setNumWorkers(options.getNumWorkers());
-
-      if (options.isStreaming()) {
-        // Use separate data disk for streaming.
-        Disk disk = new Disk();
-        disk.setDiskType(options.getWorkerDiskType());
-        workerPool.setDataDisks(Collections.singletonList(disk));
-      }
-      if (!Strings.isNullOrEmpty(options.getZone())) {
-        workerPool.setZone(options.getZone());
-      }
-      if (!Strings.isNullOrEmpty(options.getNetwork())) {
-        workerPool.setNetwork(options.getNetwork());
-      }
-      if (!Strings.isNullOrEmpty(options.getSubnetwork())) {
-        workerPool.setSubnetwork(options.getSubnetwork());
-      }
-      if (options.getDiskSizeGb() > 0) {
-        workerPool.setDiskSizeGb(options.getDiskSizeGb());
-      }
-      AutoscalingSettings settings = new AutoscalingSettings();
-      if (options.getAutoscalingAlgorithm() != null) {
-        settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm());
-      }
-      settings.setMaxNumWorkers(options.getMaxNumWorkers());
-      workerPool.setAutoscalingSettings(settings);
-
-      List<WorkerPool> workerPools = new LinkedList<>();
-
-      workerPools.add(workerPool);
-      environment.setWorkerPools(workerPools);
-
-      pipeline.traverseTopologically(this);
-      return job;
-    }
-
-    @Override
-    public DataflowPipelineOptions getPipelineOptions() {
-      return options;
-    }
-
-    @Override
-    public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
-      return (InputT) getCurrentTransform(transform).getInput();
-    }
-
-    @Override
-    public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
-      return (OutputT) getCurrentTransform(transform).getOutput();
-    }
-
-    @Override
-    public String getFullName(PTransform<?, ?> transform) {
-      return getCurrentTransform(transform).getFullName();
-    }
-
-    private AppliedPTransform<?, ?, ?> getCurrentTransform(PTransform<?, ?> transform) {
-      checkArgument(
-          currentTransform != null && currentTransform.getTransform() == transform,
-          "can only be called with current transform");
-      return currentTransform;
-    }
-
-    @Override
-    public void enterCompositeTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
-    }
-
-    @Override
-    public void visitTransform(TransformTreeNode node) {
-      PTransform<?, ?> transform = node.getTransform();
-      TransformTranslator translator =
-          getTransformTranslator(transform.getClass());
-      if (translator == null) {
-        throw new IllegalStateException(
-            "no translator registered for " + transform);
-      }
-      LOG.debug("Translating {}", transform);
-      currentTransform = AppliedPTransform.of(
-          node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
-      translator.translate(transform, this);
-      currentTransform = null;
-    }
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
-      LOG.debug("Checking translation of {}", value);
-      if (value.getProducingTransformInternal() == null) {
-        throw new RuntimeException(
-            "internal error: expecting a PValue "
-            + "to have a producingTransform");
-      }
-      if (!producer.isCompositeNode()) {
-        // Primitive transforms are the only ones assigned step names.
-        asOutputReference(value);
-      }
-    }
-
-    @Override
-    public void addStep(PTransform<?, ?> transform, String type) {
-      String stepName = genStepName();
-      if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
-        throw new IllegalArgumentException(
-            transform + " already has a name specified");
-      }
-      // Start the next "steps" list item.
-      List<Step> steps = job.getSteps();
-      if (steps == null) {
-        steps = new LinkedList<>();
-        job.setSteps(steps);
-      }
-
-      currentStep = new Step();
-      currentStep.setName(stepName);
-      currentStep.setKind(type);
-      steps.add(currentStep);
-      addInput(PropertyNames.USER_NAME, getFullName(transform));
-      addDisplayData(PropertyNames.DISPLAY_DATA, DisplayData.from(transform));
-    }
-
-    @Override
-    public void addStep(PTransform<?, ? extends PValue> transform, Step original) {
-      Step step = original.clone();
-      String stepName = step.getName();
-      if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
-        throw new IllegalArgumentException(transform + " already has a name specified");
-      }
-
-      Map<String, Object> properties = step.getProperties();
-      if (properties != null) {
-        @Nullable List<Map<String, Object>> outputInfoList = null;
-        try {
-          // TODO: This should be done via a Structs accessor.
-          @Nullable List<Map<String, Object>> list =
-              (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
-          outputInfoList = list;
-        } catch (Exception e) {
-          throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
-        }
-        if (outputInfoList != null && outputInfoList.size() > 0) {
-          Map<String, Object> firstOutputPort = outputInfoList.get(0);
-          @Nullable String name;
-          try {
-            name = getString(firstOutputPort, PropertyNames.OUTPUT_NAME);
-          } catch (Exception e) {
-            name = null;
-          }
-          if (name != null) {
-            registerOutputName(getOutput(transform), name);
-          }
-        }
-      }
-
-      List<Step> steps = job.getSteps();
-      if (steps == null) {
-        steps = new LinkedList<>();
-        job.setSteps(steps);
-      }
-      currentStep = step;
-      steps.add(step);
-    }
-
-    @Override
-    public void addEncodingInput(Coder<?> coder) {
-      CloudObject encoding = SerializableUtils.ensureSerializable(coder);
-      addObject(getProperties(), PropertyNames.ENCODING, encoding);
-    }
-
-    @Override
-    public void addInput(String name, Boolean value) {
-      addBoolean(getProperties(), name, value);
-    }
-
-    @Override
-    public void addInput(String name, String value) {
-      addString(getProperties(), name, value);
-    }
-
-    @Override
-    public void addInput(String name, Long value) {
-      addLong(getProperties(), name, value);
-    }
-
-    @Override
-    public void addInput(String name, Map<String, Object> elements) {
-      addDictionary(getProperties(), name, elements);
-    }
-
-    @Override
-    public void addInput(String name, List<? extends Map<String, Object>> elements) {
-      addList(getProperties(), name, elements);
-    }
-
-    @Override
-    public void addInput(String name, PInput value) {
-      if (value instanceof PValue) {
-        addInput(name, asOutputReference((PValue) value));
-      } else {
-        throw new IllegalStateException("Input must be a PValue");
-      }
-    }
-
-    @Override
-    public void addOutput(String name, PValue value) {
-      Coder<?> coder;
-      if (value instanceof TypedPValue) {
-        coder = ((TypedPValue<?>) value).getCoder();
-        if (value instanceof PCollection) {
-          // Wrap the PCollection element Coder inside a WindowedValueCoder.
-          coder = WindowedValue.getFullCoder(
-              coder,
-              ((PCollection<?>) value).getWindowingStrategy().getWindowFn().windowCoder());
-        }
-      } else {
-        // No output coder to encode.
-        coder = null;
-      }
-      addOutput(name, value, coder);
-    }
-
-    @Override
-    public void addValueOnlyOutput(String name, PValue value) {
-      Coder<?> coder;
-      if (value instanceof TypedPValue) {
-        coder = ((TypedPValue<?>) value).getCoder();
-        if (value instanceof PCollection) {
-          // Wrap the PCollection element Coder inside a ValueOnly
-          // WindowedValueCoder.
-          coder = WindowedValue.getValueOnlyCoder(coder);
-        }
-      } else {
-        // No output coder to encode.
-        coder = null;
-      }
-      addOutput(name, value, coder);
-    }
-
-    @Override
-    public void addCollectionToSingletonOutput(String name,
-                                               PValue inputValue,
-                                               PValue outputValue) {
-      Coder<?> inputValueCoder =
-          Preconditions.checkNotNull(outputCoders.get(inputValue));
-      // The inputValueCoder for the input PCollection should be some
-      // WindowedValueCoder of the input PCollection's element
-      // coder.
-      Preconditions.checkState(
-          inputValueCoder instanceof WindowedValue.WindowedValueCoder);
-      // The outputValueCoder for the output should be an
-      // IterableCoder of the inputValueCoder. This is a property
-      // of the backend "CollectionToSingleton" step.
-      Coder<?> outputValueCoder = IterableCoder.of(inputValueCoder);
-      addOutput(name, outputValue, outputValueCoder);
-    }
-
-    /**
-     * Adds an output with the given name to the previously added
-     * Dataflow step, producing the specified output {@code PValue}
-     * with the given {@code Coder} (if not {@code null}).
-     */
-    private void addOutput(String name, PValue value, Coder<?> valueCoder) {
-      registerOutputName(value, name);
-
-      Map<String, Object> properties = getProperties();
-      @Nullable List<Map<String, Object>> outputInfoList = null;
-      try {
-        // TODO: This should be done via a Structs accessor.
-        outputInfoList = (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
-      } catch (Exception e) {
-        throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
-      }
-      if (outputInfoList == null) {
-        outputInfoList = new ArrayList<>();
-        // TODO: This should be done via a Structs accessor.
-        properties.put(PropertyNames.OUTPUT_INFO, outputInfoList);
-      }
-
-      Map<String, Object> outputInfo = new HashMap<>();
-      addString(outputInfo, PropertyNames.OUTPUT_NAME, name);
-      addString(outputInfo, PropertyNames.USER_NAME, value.getName());
-      if (value instanceof PCollection
-          && runner.doesPCollectionRequireIndexedFormat((PCollection<?>) value)) {
-        addBoolean(outputInfo, PropertyNames.USE_INDEXED_FORMAT, true);
-      }
-      if (valueCoder != null) {
-        // Verify that encoding can be decoded, in order to catch serialization
-        // failures as early as possible.
-        CloudObject encoding = SerializableUtils.ensureSerializable(valueCoder);
-        addObject(outputInfo, PropertyNames.ENCODING, encoding);
-        outputCoders.put(value, valueCoder);
-      }
-
-      outputInfoList.add(outputInfo);
-    }
-
-    private void addDisplayData(String name, DisplayData displayData) {
-      List<Map<String, Object>> list = MAPPER.convertValue(displayData, List.class);
-      addList(getProperties(), name, list);
-    }
-
-    @Override
-    public OutputReference asOutputReference(PValue value) {
-      AppliedPTransform<?, ?, ?> transform =
-          value.getProducingTransformInternal();
-      String stepName = stepNames.get(transform);
-      if (stepName == null) {
-        throw new IllegalArgumentException(transform + " doesn't have a name specified");
-      }
-
-      String outputName = outputNames.get(value);
-      if (outputName == null) {
-        throw new IllegalArgumentException(
-            "output " + value + " doesn't have a name specified");
-      }
-
-      return new OutputReference(stepName, outputName);
-    }
-
-    private Map<String, Object> getProperties() {
-      Map<String, Object> properties = currentStep.getProperties();
-      if (properties == null) {
-        properties = new HashMap<>();
-        currentStep.setProperties(properties);
-      }
-      return properties;
-    }
-
-    /**
-     * Returns a fresh Dataflow step name.
-     */
-    private String genStepName() {
-      return "s" + (stepNames.size() + 1);
-    }
-
-    /**
-     * Records the name of the given output PValue,
-     * within its producing transform.
-     */
-    private void registerOutputName(POutput value, String name) {
-      if (outputNames.put(value, name) != null) {
-        throw new IllegalArgumentException(
-            "output " + value + " already has a name specified");
-      }
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  @Override
-  public String toString() {
-    return "DataflowPipelineTranslator#" + hashCode();
-  }
-
-
-  ///////////////////////////////////////////////////////////////////////////
-
-  static {
-    registerTransformTranslator(
-        View.CreatePCollectionView.class,
-        new TransformTranslator<View.CreatePCollectionView>() {
-          @Override
-          public void translate(
-              View.CreatePCollectionView transform,
-              TranslationContext context) {
-            translateTyped(transform, context);
-          }
-
-          private <ElemT, ViewT> void translateTyped(
-              View.CreatePCollectionView<ElemT, ViewT> transform,
-              TranslationContext context) {
-            context.addStep(transform, "CollectionToSingleton");
-            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-            context.addCollectionToSingletonOutput(
-                PropertyNames.OUTPUT,
-                context.getInput(transform),
-                context.getOutput(transform));
-          }
-        });
-
-    DataflowPipelineTranslator.registerTransformTranslator(
-        Combine.GroupedValues.class,
-        new DataflowPipelineTranslator.TransformTranslator<Combine.GroupedValues>() {
-          @Override
-          public void translate(
-              Combine.GroupedValues transform,
-              DataflowPipelineTranslator.TranslationContext context) {
-            translateHelper(transform, context);
-          }
-
-          private <K, InputT, OutputT> void translateHelper(
-              final Combine.GroupedValues<K, InputT, OutputT> transform,
-              DataflowPipelineTranslator.TranslationContext context) {
-            context.addStep(transform, "CombineValues");
-            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
-
-            AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn =
-                transform.getAppliedFn(
-                    context.getInput(transform).getPipeline().getCoderRegistry(),
-                context.getInput(transform).getCoder(),
-                context.getInput(transform).getWindowingStrategy());
-
-            context.addEncodingInput(fn.getAccumulatorCoder());
-            context.addInput(
-                PropertyNames.SERIALIZED_FN,
-                byteArrayToJsonString(serializeToByteArray(fn)));
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-          }
-        });
-
-    registerTransformTranslator(
-        Create.Values.class,
-        new TransformTranslator<Create.Values>() {
-          @Override
-          public void translate(
-              Create.Values transform,
-              TranslationContext context) {
-            createHelper(transform, context);
-          }
-
-          private <T> void createHelper(
-              Create.Values<T> transform,
-              TranslationContext context) {
-            context.addStep(transform, "CreateCollection");
-
-            Coder<T> coder = context.getOutput(transform).getCoder();
-            List<CloudObject> elements = new LinkedList<>();
-            for (T elem : transform.getElements()) {
-              byte[] encodedBytes;
-              try {
-                encodedBytes = encodeToByteArray(coder, elem);
-              } catch (CoderException exn) {
-                // TODO: Put in better element printing:
-                // truncate if too long.
-                throw new IllegalArgumentException(
-                    "Unable to encode element '" + elem + "' of transform '" + transform
-                    + "' using coder '" + coder + "'.",
-                    exn);
-              }
-              String encodedJson = byteArrayToJsonString(encodedBytes);
-              assert Arrays.equals(encodedBytes,
-                                   jsonStringToByteArray(encodedJson));
-              elements.add(CloudObject.forString(encodedJson));
-            }
-            context.addInput(PropertyNames.ELEMENT, elements);
-            context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-          }
-        });
-
-    registerTransformTranslator(
-        Flatten.FlattenPCollectionList.class,
-        new TransformTranslator<Flatten.FlattenPCollectionList>() {
-          @Override
-          public void translate(
-              Flatten.FlattenPCollectionList transform,
-              TranslationContext context) {
-            flattenHelper(transform, context);
-          }
-
-          private <T> void flattenHelper(
-              Flatten.FlattenPCollectionList<T> transform,
-              TranslationContext context) {
-            context.addStep(transform, "Flatten");
-
-            List<OutputReference> inputs = new LinkedList<>();
-            for (PCollection<T> input : context.getInput(transform).getAll()) {
-              inputs.add(context.asOutputReference(input));
-            }
-            context.addInput(PropertyNames.INPUTS, inputs);
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-          }
-        });
-
-    registerTransformTranslator(
-        GroupByKeyAndSortValuesOnly.class,
-        new TransformTranslator<GroupByKeyAndSortValuesOnly>() {
-          @Override
-          public void translate(
-              GroupByKeyAndSortValuesOnly transform,
-              TranslationContext context) {
-            groupByKeyAndSortValuesHelper(transform, context);
-          }
-
-          private <K1, K2, V> void groupByKeyAndSortValuesHelper(
-              GroupByKeyAndSortValuesOnly<K1, K2, V> transform,
-              TranslationContext context) {
-            context.addStep(transform, "GroupByKey");
-            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-            context.addInput(PropertyNames.SORT_VALUES, true);
-
-            // TODO: Add support for combiner lifting once the need arises.
-            context.addInput(
-                PropertyNames.DISALLOW_COMBINER_LIFTING, true);
-          }
-        });
-
-    registerTransformTranslator(
-        GroupByKey.class,
-        new TransformTranslator<GroupByKey>() {
-          @Override
-          public void translate(
-              GroupByKey transform,
-              TranslationContext context) {
-            groupByKeyHelper(transform, context);
-          }
-
-          private <K, V> void groupByKeyHelper(
-              GroupByKey<K, V> transform,
-              TranslationContext context) {
-            context.addStep(transform, "GroupByKey");
-            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-
-            WindowingStrategy<?, ?> windowingStrategy =
-                context.getInput(transform).getWindowingStrategy();
-            boolean isStreaming =
-                context.getPipelineOptions().as(StreamingOptions.class).isStreaming();
-            boolean disallowCombinerLifting =
-                !windowingStrategy.getWindowFn().isNonMerging()
-                || (isStreaming && !transform.fewKeys())
-                // TODO: Allow combiner lifting on the non-default trigger, as appropriate.
-                || !(windowingStrategy.getTrigger().getSpec() instanceof DefaultTrigger);
-            context.addInput(
-                PropertyNames.DISALLOW_COMBINER_LIFTING, disallowCombinerLifting);
-            context.addInput(
-                PropertyNames.SERIALIZED_FN,
-                byteArrayToJsonString(serializeToByteArray(windowingStrategy)));
-            context.addInput(
-                PropertyNames.IS_MERGING_WINDOW_FN,
-                !windowingStrategy.getWindowFn().isNonMerging());
-          }
-        });
-
-    registerTransformTranslator(
-        ParDo.BoundMulti.class,
-        new TransformTranslator<ParDo.BoundMulti>() {
-          @Override
-          public void translate(
-              ParDo.BoundMulti transform,
-              TranslationContext context) {
-            translateMultiHelper(transform, context);
-          }
-
-          private <InputT, OutputT> void translateMultiHelper(
-              ParDo.BoundMulti<InputT, OutputT> transform,
-              TranslationContext context) {
-            context.addStep(transform, "ParallelDo");
-            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
-            translateFn(transform.getFn(), context.getInput(transform).getWindowingStrategy(),
-                transform.getSideInputs(), context.getInput(transform).getCoder(), context);
-            translateOutputs(context.getOutput(transform), context);
-          }
-        });
-
-    registerTransformTranslator(
-        ParDo.Bound.class,
-        new TransformTranslator<ParDo.Bound>() {
-          @Override
-          public void translate(
-              ParDo.Bound transform,
-              TranslationContext context) {
-            translateSingleHelper(transform, context);
-          }
-
-          private <InputT, OutputT> void translateSingleHelper(
-              ParDo.Bound<InputT, OutputT> transform,
-              TranslationContext context) {
-            context.addStep(transform, "ParallelDo");
-            translateInputs(context.getInput(transform), transform.getSideInputs(), context);
-            translateFn(
-                transform.getFn(),
-                context.getInput(transform).getWindowingStrategy(),
-                transform.getSideInputs(), context.getInput(transform).getCoder(), context);
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-          }
-        });
-
-    registerTransformTranslator(
-        Window.Bound.class,
-        new DataflowPipelineTranslator.TransformTranslator<Window.Bound>() {
-          @Override
-          public void translate(
-              Window.Bound transform, TranslationContext context) {
-            translateHelper(transform, context);
-          }
-
-          private <T> void translateHelper(
-              Window.Bound<T> transform, TranslationContext context) {
-            context.addStep(transform, "Bucket");
-            context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-            context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-
-            WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy();
-            byte[] serializedBytes = serializeToByteArray(strategy);
-            String serializedJson = byteArrayToJsonString(serializedBytes);
-            assert Arrays.equals(serializedBytes,
-                                 jsonStringToByteArray(serializedJson));
-            context.addInput(PropertyNames.SERIALIZED_FN, serializedJson);
-          }
-        });
-
-    ///////////////////////////////////////////////////////////////////////////
-    // IO Translation.
-
-    registerTransformTranslator(
-        BigQueryIO.Read.Bound.class, new BigQueryIOTranslator.ReadTranslator());
-    registerTransformTranslator(
-        BigQueryIO.Write.Bound.class, new BigQueryIOTranslator.WriteTranslator());
-
-    registerTransformTranslator(
-        PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator());
-    registerTransformTranslator(
-        DataflowPipelineRunner.StreamingPubsubIOWrite.class,
-        new PubsubIOTranslator.WriteTranslator());
-
-    registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
-  }
-
-  private static void translateInputs(
-      PCollection<?> input,
-      List<PCollectionView<?>> sideInputs,
-      TranslationContext context) {
-    context.addInput(PropertyNames.PARALLEL_INPUT, input);
-    translateSideInputs(sideInputs, context);
-  }
-
-  // Used for ParDo
-  private static void translateSideInputs(
-      List<PCollectionView<?>> sideInputs,
-      TranslationContext context) {
-    Map<String, Object> nonParInputs = new HashMap<>();
-
-    for (PCollectionView<?> view : sideInputs) {
-      nonParInputs.put(
-          view.getTagInternal().getId(),
-          context.asOutputReference(view));
-    }
-
-    context.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs);
-  }
-
-  private static void translateFn(
-      DoFn fn,
-      WindowingStrategy windowingStrategy,
-      Iterable<PCollectionView<?>> sideInputs,
-      Coder inputCoder,
-      TranslationContext context) {
-    context.addInput(PropertyNames.USER_FN, fn.getClass().getName());
-    context.addInput(
-        PropertyNames.SERIALIZED_FN,
-        byteArrayToJsonString(serializeToByteArray(
-            new DoFnInfo(fn, windowingStrategy, sideInputs, inputCoder))));
-  }
-
-  private static void translateOutputs(
-      PCollectionTuple outputs,
-      TranslationContext context) {
-    for (Map.Entry<TupleTag<?>, PCollection<?>> entry
-             : outputs.getAll().entrySet()) {
-      TupleTag<?> tag = entry.getKey();
-      PCollection<?> output = entry.getValue();
-      context.addOutput(tag.getId(), output);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowServiceException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowServiceException.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowServiceException.java
deleted file mode 100644
index 8b9f3f4..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowServiceException.java
+++ /dev/null
@@ -1,33 +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 com.google.cloud.dataflow.sdk.runners;
-
-import javax.annotation.Nullable;
-
-/**
- * Signals there was an error retrieving information about a job from the Cloud Dataflow Service.
- */
-public class DataflowServiceException extends DataflowJobException {
-  DataflowServiceException(DataflowPipelineJob job, String message) {
-    this(job, message, null);
-  }
-
-  DataflowServiceException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
-    super(job, message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AssignWindows.java
deleted file mode 100644
index 974c3a9..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AssignWindows.java
+++ /dev/null
@@ -1,90 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-/**
- * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)}
- * {@link PTransform}.
- *
- * For an application of {@link Window#into(WindowFn)} that changes the {@link WindowFn}, applies
- * a primitive {@link PTransform} in the Dataflow service.
- *
- * For an application of {@link Window#into(WindowFn)} that does not change the {@link WindowFn},
- * applies an identity {@link ParDo} and sets the windowing strategy of the output
- * {@link PCollection}.
- *
- * For internal use only.
- *
- * @param <T> the type of input element
- */
-public class AssignWindows<T> extends PTransform<PCollection<T>, PCollection<T>> {
-  private final Window.Bound<T> transform;
-
-  /**
-   * Builds an instance of this class from the overriden transform.
-   */
-  @SuppressWarnings("unused") // Used via reflection
-  public AssignWindows(Window.Bound<T> transform) {
-    this.transform = transform;
-  }
-
-  @Override
-  public PCollection<T> apply(PCollection<T> input) {
-    WindowingStrategy<?, ?> outputStrategy =
-        transform.getOutputStrategyInternal(input.getWindowingStrategy());
-    if (transform.getWindowFn() != null) {
-      // If the windowFn changed, we create a primitive, and run the AssignWindows operation here.
-      return PCollection.<T>createPrimitiveOutputInternal(
-                            input.getPipeline(), outputStrategy, input.isBounded());
-    } else {
-      // If the windowFn didn't change, we just run a pass-through transform and then set the
-      // new windowing strategy.
-      return input.apply(ParDo.named("Identity").of(new DoFn<T, T>() {
-        @Override
-        public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
-          c.output(c.element());
-        }
-      })).setWindowingStrategyInternal(outputStrategy);
-    }
-  }
-
-  @Override
-  public void validate(PCollection<T> input) {
-    transform.validate(input);
-  }
-
-  @Override
-  protected Coder<?> getDefaultOutputCoder(PCollection<T> input) {
-    return input.getCoder();
-  }
-
-  @Override
-  protected String getKindString() {
-    return "Window.Into()";
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java
deleted file mode 100755
index 203ce4f..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java
+++ /dev/null
@@ -1,126 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import com.google.api.client.json.JsonFactory;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.Transport;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * BigQuery transform support code for the Dataflow backend.
- */
-public class BigQueryIOTranslator {
-  private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
-  private static final Logger LOG = LoggerFactory.getLogger(BigQueryIOTranslator.class);
-
-  /**
-   * Implements BigQueryIO Read translation for the Dataflow backend.
-   */
-  public static class ReadTranslator
-      implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Read.Bound> {
-
-    @Override
-    public void translate(
-        BigQueryIO.Read.Bound transform, DataflowPipelineTranslator.TranslationContext context) {
-      // Actual translation.
-      context.addStep(transform, "ParallelRead");
-      context.addInput(PropertyNames.FORMAT, "bigquery");
-      context.addInput(PropertyNames.BIGQUERY_EXPORT_FORMAT, "FORMAT_AVRO");
-
-      if (transform.getQuery() != null) {
-        context.addInput(PropertyNames.BIGQUERY_QUERY, transform.getQuery());
-        context.addInput(PropertyNames.BIGQUERY_FLATTEN_RESULTS, transform.getFlattenResults());
-      } else {
-        TableReference table = transform.getTable();
-        if (table.getProjectId() == null) {
-          // If user does not specify a project we assume the table to be located in the project
-          // that owns the Dataflow job.
-          String projectIdFromOptions = context.getPipelineOptions().getProject();
-          LOG.warn(String.format(BigQueryIO.SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
-              table.getDatasetId(), table.getTableId(), projectIdFromOptions));
-          table.setProjectId(projectIdFromOptions);
-        }
-
-        context.addInput(PropertyNames.BIGQUERY_TABLE, table.getTableId());
-        context.addInput(PropertyNames.BIGQUERY_DATASET, table.getDatasetId());
-        if (table.getProjectId() != null) {
-          context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
-        }
-      }
-      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-    }
-  }
-
-  /**
-   * Implements BigQueryIO Write translation for the Dataflow backend.
-   */
-  public static class WriteTranslator
-      implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Write.Bound> {
-
-    @Override
-    public void translate(BigQueryIO.Write.Bound transform,
-                          DataflowPipelineTranslator.TranslationContext context) {
-      if (context.getPipelineOptions().isStreaming()) {
-        // Streaming is handled by the streaming runner.
-        throw new AssertionError(
-            "BigQueryIO is specified to use streaming write in batch mode.");
-      }
-
-      TableReference table = transform.getTable();
-
-      // Actual translation.
-      context.addStep(transform, "ParallelWrite");
-      context.addInput(PropertyNames.FORMAT, "bigquery");
-      context.addInput(PropertyNames.BIGQUERY_TABLE,
-                       table.getTableId());
-      context.addInput(PropertyNames.BIGQUERY_DATASET,
-                       table.getDatasetId());
-      if (table.getProjectId() != null) {
-        context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
-      }
-      if (transform.getSchema() != null) {
-        try {
-          context.addInput(PropertyNames.BIGQUERY_SCHEMA,
-                           JSON_FACTORY.toString(transform.getSchema()));
-        } catch (IOException exn) {
-          throw new IllegalArgumentException("Invalid table schema.", exn);
-        }
-      }
-      context.addInput(
-          PropertyNames.BIGQUERY_CREATE_DISPOSITION,
-          transform.getCreateDisposition().name());
-      context.addInput(
-          PropertyNames.BIGQUERY_WRITE_DISPOSITION,
-          transform.getWriteDisposition().name());
-      // Set sink encoding to TableRowJsonCoder.
-      context.addEncodingInput(
-          WindowedValue.getValueOnlyCoder(TableRowJsonCoder.of()));
-      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java
deleted file mode 100755
index 3473c41..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java
+++ /dev/null
@@ -1,119 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import static com.google.api.client.util.Base64.encodeBase64String;
-import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
-import static com.google.cloud.dataflow.sdk.util.Structs.addStringList;
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.services.dataflow.model.SourceMetadata;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-
-/**
- * A helper class for supporting sources defined as {@code Source}.
- *
- * <p>Provides a bridge between the high-level {@code Source} API and the
- * low-level {@code CloudSource} class.
- */
-public class CustomSources {
-  private static final String SERIALIZED_SOURCE = "serialized_source";
-  @VisibleForTesting static final String SERIALIZED_SOURCE_SPLITS = "serialized_source_splits";
-  /**
-   * The current limit on the size of a ReportWorkItemStatus RPC to Google Cloud Dataflow, which
-   * includes the initial splits, is 20 MB.
-   */
-  public static final long DATAFLOW_SPLIT_RESPONSE_API_SIZE_BYTES = 20 * (1 << 20);
-
-  private static final Logger LOG = LoggerFactory.getLogger(CustomSources.class);
-
-  private static final ByteString firstSplitKey = ByteString.copyFromUtf8("0000000000000001");
-
-  public static boolean isFirstUnboundedSourceSplit(ByteString splitKey) {
-    return splitKey.equals(firstSplitKey);
-  }
-
-  private static int getDesiredNumUnboundedSourceSplits(DataflowPipelineOptions options) {
-    if (options.getMaxNumWorkers() > 0) {
-      return options.getMaxNumWorkers();
-    } else if (options.getNumWorkers() > 0) {
-      return options.getNumWorkers() * 3;
-    } else {
-      return 20;
-    }
-  }
-
-  public static com.google.api.services.dataflow.model.Source serializeToCloudSource(
-      Source<?> source, PipelineOptions options) throws Exception {
-    com.google.api.services.dataflow.model.Source cloudSource =
-        new com.google.api.services.dataflow.model.Source();
-    // We ourselves act as the SourceFormat.
-    cloudSource.setSpec(CloudObject.forClass(CustomSources.class));
-    addString(
-        cloudSource.getSpec(), SERIALIZED_SOURCE, encodeBase64String(serializeToByteArray(source)));
-
-    SourceMetadata metadata = new SourceMetadata();
-    if (source instanceof BoundedSource) {
-      BoundedSource<?> boundedSource = (BoundedSource<?>) source;
-      try {
-        metadata.setProducesSortedKeys(boundedSource.producesSortedKeys(options));
-      } catch (Exception e) {
-        LOG.warn("Failed to check if the source produces sorted keys: " + source, e);
-      }
-
-      // Size estimation is best effort so we continue even if it fails here.
-      try {
-        metadata.setEstimatedSizeBytes(boundedSource.getEstimatedSizeBytes(options));
-      } catch (Exception e) {
-        LOG.warn("Size estimation of the source failed: " + source, e);
-      }
-    } else if (source instanceof UnboundedSource) {
-      UnboundedSource<?, ?> unboundedSource = (UnboundedSource<?, ?>) source;
-      metadata.setInfinite(true);
-      List<String> encodedSplits = new ArrayList<>();
-      int desiredNumSplits =
-          getDesiredNumUnboundedSourceSplits(options.as(DataflowPipelineOptions.class));
-      for (UnboundedSource<?, ?> split :
-          unboundedSource.generateInitialSplits(desiredNumSplits, options)) {
-        encodedSplits.add(encodeBase64String(serializeToByteArray(split)));
-      }
-      checkArgument(!encodedSplits.isEmpty(), "UnboundedSources must have at least one split");
-      addStringList(cloudSource.getSpec(), SERIALIZED_SOURCE_SPLITS, encodedSplits);
-    } else {
-      throw new IllegalArgumentException("Unexpected source kind: " + source.getClass());
-    }
-
-    cloudSource.setMetadata(metadata);
-    return cloudSource;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowAggregatorTransforms.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowAggregatorTransforms.java
deleted file mode 100755
index 5c1dd95..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowAggregatorTransforms.java
+++ /dev/null
@@ -1,80 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-
-/**
- * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used.
- */
-public class DataflowAggregatorTransforms {
-  private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms;
-  private final Multimap<PTransform<?, ?>, AppliedPTransform<?, ?, ?>> transformAppliedTransforms;
-  private final BiMap<AppliedPTransform<?, ?, ?>, String> appliedStepNames;
-
-  public DataflowAggregatorTransforms(
-      Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms,
-      Map<AppliedPTransform<?, ?, ?>, String> transformStepNames) {
-    this.aggregatorTransforms = aggregatorTransforms;
-    appliedStepNames = HashBiMap.create(transformStepNames);
-
-    transformAppliedTransforms = HashMultimap.create();
-    for (AppliedPTransform<?, ?, ?> appliedTransform : transformStepNames.keySet()) {
-      transformAppliedTransforms.put(appliedTransform.getTransform(), appliedTransform);
-    }
-  }
-
-  /**
-   * Returns true if the provided {@link Aggregator} is used in the constructing {@link Pipeline}.
-   */
-  public boolean contains(Aggregator<?, ?> aggregator) {
-    return aggregatorTransforms.containsKey(aggregator);
-  }
-
-  /**
-   * Gets the step names in which the {@link Aggregator} is used.
-   */
-  public Collection<String> getAggregatorStepNames(Aggregator<?, ?> aggregator) {
-    Collection<String> names = new HashSet<>();
-    Collection<PTransform<?, ?>> transforms = aggregatorTransforms.get(aggregator);
-    for (PTransform<?, ?> transform : transforms) {
-      for (AppliedPTransform<?, ?, ?> applied : transformAppliedTransforms.get(transform)) {
-        names.add(appliedStepNames.get(applied));
-      }
-    }
-    return names;
-  }
-
-  /**
-   * Gets the {@link PTransform} that was assigned the provided step name.
-   */
-  public AppliedPTransform<?, ?, ?> getAppliedTransformForStepName(String stepName) {
-    return appliedStepNames.inverse().get(stepName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
deleted file mode 100755
index 8de1038..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
+++ /dev/null
@@ -1,111 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Methods for extracting the values of an {@link Aggregator} from a collection of {@link
- * MetricUpdate MetricUpdates}.
- */
-public final class DataflowMetricUpdateExtractor {
-  private static final String STEP_NAME_CONTEXT_KEY = "step";
-  private static final String IS_TENTATIVE_KEY = "tentative";
-
-  private DataflowMetricUpdateExtractor() {
-    // Do not instantiate.
-  }
-
-  /**
-   * Extract the values of the provided {@link Aggregator} at each {@link PTransform} it was used in
-   * according to the provided {@link DataflowAggregatorTransforms} from the given list of {@link
-   * MetricUpdate MetricUpdates}.
-   */
-  public static <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator,
-      DataflowAggregatorTransforms aggregatorTransforms, List<MetricUpdate> metricUpdates) {
-    Map<String, OutputT> results = new HashMap<>();
-    if (metricUpdates == null) {
-      return results;
-    }
-
-    String aggregatorName = aggregator.getName();
-    Collection<String> aggregatorSteps = aggregatorTransforms.getAggregatorStepNames(aggregator);
-
-    for (MetricUpdate metricUpdate : metricUpdates) {
-      MetricStructuredName metricStructuredName = metricUpdate.getName();
-      Map<String, String> context = metricStructuredName.getContext();
-      if (metricStructuredName.getName().equals(aggregatorName) && context != null
-          && aggregatorSteps.contains(context.get(STEP_NAME_CONTEXT_KEY))) {
-        AppliedPTransform<?, ?, ?> transform =
-            aggregatorTransforms.getAppliedTransformForStepName(
-                context.get(STEP_NAME_CONTEXT_KEY));
-        String fullName = transform.getFullName();
-        // Prefer the tentative (fresher) value if it exists.
-        if (Boolean.parseBoolean(context.get(IS_TENTATIVE_KEY)) || !results.containsKey(fullName)) {
-          results.put(fullName, toValue(aggregator, metricUpdate));
-        }
-      }
-    }
-
-    return results;
-
-  }
-
-  private static <OutputT> OutputT toValue(
-      Aggregator<?, OutputT> aggregator, MetricUpdate metricUpdate) {
-    CombineFn<?, ?, OutputT> combineFn = aggregator.getCombineFn();
-    Class<? super OutputT> outputType = combineFn.getOutputType().getRawType();
-
-    if (outputType.equals(Long.class)) {
-      @SuppressWarnings("unchecked")
-      OutputT asLong = (OutputT) Long.valueOf(toNumber(metricUpdate).longValue());
-      return asLong;
-    }
-    if (outputType.equals(Integer.class)) {
-      @SuppressWarnings("unchecked")
-      OutputT asInt = (OutputT) Integer.valueOf(toNumber(metricUpdate).intValue());
-      return asInt;
-    }
-    if (outputType.equals(Double.class)) {
-      @SuppressWarnings("unchecked")
-      OutputT asDouble = (OutputT) Double.valueOf(toNumber(metricUpdate).doubleValue());
-      return asDouble;
-    }
-    throw new UnsupportedOperationException(
-        "Unsupported Output Type " + outputType + " in aggregator " + aggregator);
-  }
-
-  private static Number toNumber(MetricUpdate update) {
-    if (update.getScalar() instanceof Number) {
-      return (Number) update.getScalar();
-    }
-    throw new IllegalArgumentException(
-        "Metric Update " + update + " does not have a numeric scalar");
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java
deleted file mode 100755
index b54d5c6..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java
+++ /dev/null
@@ -1,108 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-
-/**
- * Pubsub transform support code for the Dataflow backend.
- */
-public class PubsubIOTranslator {
-
-  /**
-   * Implements PubsubIO Read translation for the Dataflow backend.
-   */
-  public static class ReadTranslator<T> implements TransformTranslator<PubsubIO.Read.Bound<T>> {
-    @Override
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    public void translate(
-        PubsubIO.Read.Bound transform,
-        TranslationContext context) {
-      translateReadHelper(transform, context);
-    }
-
-    private <T> void translateReadHelper(
-        PubsubIO.Read.Bound<T> transform,
-        TranslationContext context) {
-      if (!context.getPipelineOptions().isStreaming()) {
-        throw new IllegalArgumentException(
-            "PubsubIO.Read can only be used with the Dataflow streaming runner.");
-      }
-
-      context.addStep(transform, "ParallelRead");
-      context.addInput(PropertyNames.FORMAT, "pubsub");
-      if (transform.getTopic() != null) {
-        context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
-      }
-      if (transform.getSubscription() != null) {
-        context.addInput(
-            PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription().asV1Beta1Path());
-      }
-      if (transform.getTimestampLabel() != null) {
-        context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
-      }
-      if (transform.getIdLabel() != null) {
-        context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
-      }
-      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-    }
-  }
-
-  /**
-   * Implements PubsubIO Write translation for the Dataflow backend.
-   */
-  public static class WriteTranslator<T>
-      implements TransformTranslator<DataflowPipelineRunner.StreamingPubsubIOWrite<T>> {
-
-    @Override
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    public void translate(
-        DataflowPipelineRunner.StreamingPubsubIOWrite transform,
-        TranslationContext context) {
-      translateWriteHelper(transform, context);
-    }
-
-    private <T> void translateWriteHelper(
-        DataflowPipelineRunner.StreamingPubsubIOWrite<T> customTransform,
-        TranslationContext context) {
-      if (!context.getPipelineOptions().isStreaming()) {
-        throw new IllegalArgumentException(
-            "PubsubIO.Write is non-primitive for the Dataflow batch runner.");
-      }
-
-      PubsubIO.Write.Bound<T> transform = customTransform.getOverriddenTransform();
-
-      context.addStep(customTransform, "ParallelWrite");
-      context.addInput(PropertyNames.FORMAT, "pubsub");
-      context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
-      if (transform.getTimestampLabel() != null) {
-        context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
-      }
-      if (transform.getIdLabel() != null) {
-        context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
-      }
-      context.addEncodingInput(WindowedValue.getValueOnlyCoder(transform.getCoder()));
-      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(customTransform));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadTranslator.java
deleted file mode 100755
index 4998af3..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadTranslator.java
+++ /dev/null
@@ -1,104 +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 com.google.cloud.dataflow.sdk.runners.dataflow;
-
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary;
-import static com.google.cloud.dataflow.sdk.util.Structs.addLong;
-
-import com.google.api.services.dataflow.model.SourceMetadata;
-import com.google.cloud.dataflow.sdk.io.FileBasedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.values.PValue;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end.
- */
-public class ReadTranslator implements TransformTranslator<Read.Bounded<?>> {
-  @Override
-  public void translate(Read.Bounded<?> transform, TranslationContext context) {
-    translateReadHelper(transform.getSource(), transform, context);
-  }
-
-  public static <T> void translateReadHelper(Source<T> source,
-      PTransform<?, ? extends PValue> transform,
-      DataflowPipelineTranslator.TranslationContext context) {
-    try {
-      // TODO: Move this validation out of translation once IOChannelUtils is portable
-      // and can be reconstructed on the worker.
-      if (source instanceof FileBasedSource) {
-        String filePatternOrSpec = ((FileBasedSource<?>) source).getFileOrPatternSpec();
-        context.getPipelineOptions()
-               .getPathValidator()
-               .validateInputFilePatternSupported(filePatternOrSpec);
-      }
-
-      context.addStep(transform, "ParallelRead");
-      context.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT);
-      context.addInput(
-          PropertyNames.SOURCE_STEP_INPUT,
-          cloudSourceToDictionary(
-              CustomSources.serializeToCloudSource(source, context.getPipelineOptions())));
-      context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  // Represents a cloud Source as a dictionary for encoding inside the {@code SOURCE_STEP_INPUT}
-  // property of CloudWorkflowStep.input.
-  private static Map<String, Object> cloudSourceToDictionary(
-      com.google.api.services.dataflow.model.Source source) {
-    // Do not translate encoding - the source's encoding is translated elsewhere
-    // to the step's output info.
-    Map<String, Object> res = new HashMap<>();
-    addDictionary(res, PropertyNames.SOURCE_SPEC, source.getSpec());
-    if (source.getMetadata() != null) {
-      addDictionary(res, PropertyNames.SOURCE_METADATA,
-          cloudSourceMetadataToDictionary(source.getMetadata()));
-    }
-    if (source.getDoesNotNeedSplitting() != null) {
-      addBoolean(
-          res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting());
-    }
-    return res;
-  }
-
-  private static Map<String, Object> cloudSourceMetadataToDictionary(SourceMetadata metadata) {
-    Map<String, Object> res = new HashMap<>();
-    if (metadata.getProducesSortedKeys() != null) {
-      addBoolean(res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys());
-    }
-    if (metadata.getEstimatedSizeBytes() != null) {
-      addLong(res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes());
-    }
-    if (metadata.getInfinite() != null) {
-      addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite());
-    }
-    return res;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java
deleted file mode 100755
index d1484ce..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Implementation of the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
- */
-package com.google.cloud.dataflow.sdk.runners.dataflow;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java
deleted file mode 100644
index 2f920a6..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java
+++ /dev/null
@@ -1,27 +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 com.google.cloud.dataflow.sdk.testing;
-
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-
-/**
- * A set of options used to configure the {@link TestPipeline}.
- */
-public interface TestDataflowPipelineOptions extends BlockingDataflowPipelineOptions {
-
-}


[25/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
new file mode 100644
index 0000000..c6dca2d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
@@ -0,0 +1,1369 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static com.google.cloud.dataflow.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.startsWith;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
+import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
+import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.VarLongCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.AvroSource;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsList;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMap;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.BatchViewAsMultimap;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner.TransformedMap;
+import com.google.cloud.dataflow.sdk.runners.dataflow.TestCountingSource;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecord;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import com.google.cloud.dataflow.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFnTester;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+import com.google.cloud.dataflow.sdk.util.UserCodeException;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TimestampedValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.hamcrest.Description;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for DataflowPipelineRunner.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineRunnerTest {
+
+  private static final String PROJECT_ID = "some-project";
+
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  // Asserts that the given Job has all expected fields set.
+  private static void assertValidJob(Job job) {
+    assertNull(job.getId());
+    assertNull(job.getCurrentState());
+  }
+
+  private DataflowPipeline buildDataflowPipeline(DataflowPipelineOptions options) {
+    options.setStableUniqueNames(CheckEnabled.ERROR);
+    DataflowPipeline p = DataflowPipeline.create(options);
+
+    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
+        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+
+    return p;
+  }
+
+  private static Dataflow buildMockDataflow(
+      final ArgumentCaptor<Job> jobCaptor) throws IOException {
+    Dataflow mockDataflowClient = mock(Dataflow.class);
+    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
+    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
+    Dataflow.Projects.Jobs.Create mockRequest =
+        mock(Dataflow.Projects.Jobs.Create.class);
+    Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
+
+    when(mockDataflowClient.projects()).thenReturn(mockProjects);
+    when(mockProjects.jobs()).thenReturn(mockJobs);
+    when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
+        .thenReturn(mockRequest);
+    when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
+    when(mockList.setPageToken(anyString())).thenReturn(mockList);
+    when(mockList.execute())
+        .thenReturn(new ListJobsResponse().setJobs(
+            Arrays.asList(new Job()
+                              .setName("oldJobName")
+                              .setId("oldJobId")
+                              .setCurrentState("JOB_STATE_RUNNING"))));
+
+    Job resultJob = new Job();
+    resultJob.setId("newid");
+    when(mockRequest.execute()).thenReturn(resultJob);
+    return mockDataflowClient;
+  }
+
+  private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
+    GcsUtil mockGcsUtil = mock(GcsUtil.class);
+    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+        .then(new Answer<SeekableByteChannel>() {
+              @Override
+              public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+                return FileChannel.open(
+                    Files.createTempFile("channel-", ".tmp"),
+                    StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+              }
+            });
+
+    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
+    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
+      @Override
+      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+      }
+    });
+    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
+    return mockGcsUtil;
+  }
+
+  private DataflowPipelineOptions buildPipelineOptions() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+    return buildPipelineOptions(jobCaptor);
+  }
+
+  private DataflowPipelineOptions buildPipelineOptions(
+      ArgumentCaptor<Job> jobCaptor) throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setProject(PROJECT_ID);
+    options.setTempLocation("gs://somebucket/some/path");
+    // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
+    options.setFilesToStage(new LinkedList<String>());
+    options.setDataflowClient(buildMockDataflow(jobCaptor));
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+    return options;
+  }
+
+  @Test
+  public void testRun() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    DataflowPipeline p = buildDataflowPipeline(options);
+    DataflowPipelineJob job = p.run();
+    assertEquals("newid", job.getJobId());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testRunReturnDifferentRequestId() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    Dataflow mockDataflowClient = options.getDataflowClient();
+    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+        .thenReturn(mockRequest);
+    Job resultJob = new Job();
+    resultJob.setId("newid");
+    // Return a different request id.
+    resultJob.setClientRequestId("different_request_id");
+    when(mockRequest.execute()).thenReturn(resultJob);
+
+    DataflowPipeline p = buildDataflowPipeline(options);
+    try {
+      p.run();
+      fail("Expected DataflowJobAlreadyExistsException");
+    } catch (DataflowJobAlreadyExistsException expected) {
+      assertThat(expected.getMessage(),
+          containsString("If you want to submit a second job, try again by setting a "
+            + "different name using --jobName."));
+      assertEquals(expected.getJob().getJobId(), resultJob.getId());
+    }
+  }
+
+  @Test
+  public void testUpdate() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setUpdate(true);
+    options.setJobName("oldJobName");
+    DataflowPipeline p = buildDataflowPipeline(options);
+    DataflowPipelineJob job = p.run();
+    assertEquals("newid", job.getJobId());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testUpdateNonExistentPipeline() throws IOException {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Could not find running job named badJobName");
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setUpdate(true);
+    options.setJobName("badJobName");
+    DataflowPipeline p = buildDataflowPipeline(options);
+    p.run();
+  }
+
+  @Test
+  public void testUpdateAlreadyUpdatedPipeline() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setUpdate(true);
+    options.setJobName("oldJobName");
+    Dataflow mockDataflowClient = options.getDataflowClient();
+    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+        .thenReturn(mockRequest);
+    final Job resultJob = new Job();
+    resultJob.setId("newid");
+    // Return a different request id.
+    resultJob.setClientRequestId("different_request_id");
+    when(mockRequest.execute()).thenReturn(resultJob);
+
+    DataflowPipeline p = buildDataflowPipeline(options);
+
+    thrown.expect(DataflowJobAlreadyUpdatedException.class);
+    thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("Expected job ID: " + resultJob.getId());
+      }
+
+      @Override
+      protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
+        return resultJob.getId().equals(item.getJob().getJobId());
+      }
+    });
+    thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
+        + "into job id: newid and cannot be updated again.");
+    p.run();
+  }
+
+  @Test
+  public void testRunWithFiles() throws IOException {
+    // Test that the function DataflowPipelineRunner.stageFiles works as
+    // expected.
+    GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
+    final String gcsStaging = "gs://somebucket/some/path";
+    final String gcsTemp = "gs://somebucket/some/temp/path";
+    final String cloudDataflowDataset = "somedataset";
+
+    // Create some temporary files.
+    File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt");
+    temp1.deleteOnExit();
+    File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt");
+    temp2.deleteOnExit();
+
+    String overridePackageName = "alias.txt";
+
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setFilesToStage(ImmutableList.of(
+        temp1.getAbsolutePath(),
+        overridePackageName + "=" + temp2.getAbsolutePath()));
+    options.setStagingLocation(gcsStaging);
+    options.setTempLocation(gcsTemp);
+    options.setTempDatasetId(cloudDataflowDataset);
+    options.setProject(PROJECT_ID);
+    options.setJobName("job");
+    options.setDataflowClient(buildMockDataflow(jobCaptor));
+    options.setGcsUtil(mockGcsUtil);
+    options.setGcpCredential(new TestCredential());
+
+    DataflowPipeline p = buildDataflowPipeline(options);
+
+    DataflowPipelineJob job = p.run();
+    assertEquals("newid", job.getJobId());
+
+    Job workflowJob = jobCaptor.getValue();
+    assertValidJob(workflowJob);
+
+    assertEquals(
+        2,
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
+    DataflowPackage workflowPackage1 =
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
+    assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
+    DataflowPackage workflowPackage2 =
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
+    assertEquals(overridePackageName, workflowPackage2.getName());
+
+    assertEquals(
+        "storage.googleapis.com/somebucket/some/temp/path",
+        workflowJob.getEnvironment().getTempStoragePrefix());
+    assertEquals(
+        cloudDataflowDataset,
+        workflowJob.getEnvironment().getDataset());
+    assertEquals(
+        DataflowReleaseInfo.getReleaseInfo().getName(),
+        workflowJob.getEnvironment().getUserAgent().get("name"));
+    assertEquals(
+        DataflowReleaseInfo.getReleaseInfo().getVersion(),
+        workflowJob.getEnvironment().getUserAgent().get("version"));
+  }
+
+  @Test
+  public void runWithDefaultFilesToStage() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setFilesToStage(null);
+    DataflowPipelineRunner.fromOptions(options);
+    assertTrue(!options.getFilesToStage().isEmpty());
+  }
+
+  @Test
+  public void detectClassPathResourceWithFileResources() throws Exception {
+    File file = tmpFolder.newFile("file");
+    File file2 = tmpFolder.newFile("file2");
+    URLClassLoader classLoader = new URLClassLoader(new URL[]{
+        file.toURI().toURL(),
+        file2.toURI().toURL()
+    });
+
+    assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
+        DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader));
+  }
+
+  @Test
+  public void detectClassPathResourcesWithUnsupportedClassLoader() {
+    ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
+
+    DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader);
+  }
+
+  @Test
+  public void detectClassPathResourceWithNonFileResources() throws Exception {
+    String url = "http://www.google.com/all-the-secrets.jar";
+    URLClassLoader classLoader = new URLClassLoader(new URL[]{
+        new URL(url)
+    });
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Unable to convert url (" + url + ") to file.");
+
+    DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader);
+  }
+
+  @Test
+  public void testGcsStagingLocationInitialization() throws Exception {
+    // Test that the staging location is initialized correctly.
+    String gcsTemp = "gs://somebucket/some/temp/path";
+
+    // Set temp location (required), and check that staging location is set.
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setTempLocation(gcsTemp);
+    options.setProject(PROJECT_ID);
+    options.setGcpCredential(new TestCredential());
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    DataflowPipelineRunner.fromOptions(options);
+
+    assertNotNull(options.getStagingLocation());
+  }
+
+  @Test
+  public void testNonGcsFilePathInReadFailure() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+    p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
+
+    thrown.expectCause(Matchers.allOf(
+        instanceOf(IllegalArgumentException.class),
+        ThrowableMessageMatcher.hasMessage(
+            containsString("expected a valid 'gs://' path but was given"))));
+    p.run();
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNonGcsFilePathInWriteFailure() throws IOException {
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").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"));
+  }
+
+  @Test
+  public void testMultiSlashGcsFileReadPath() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+    p.apply(TextIO.Read.named("ReadInvalidGcsFile")
+        .from("gs://bucket/tmp//file"));
+
+    thrown.expectCause(Matchers.allOf(
+        instanceOf(IllegalArgumentException.class),
+        ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
+    p.run();
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testMultiSlashGcsFileWritePath() throws IOException {
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("consecutive slashes");
+    pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
+  }
+
+  @Test
+  public void testInvalidTempLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setTempLocation("file://temp/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+    DataflowPipelineRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testInvalidStagingLocation() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setStagingLocation("file://my/staging/location");
+    try {
+      DataflowPipelineRunner.fromOptions(options);
+      fail("fromOptions should have failed");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+    }
+    options.setStagingLocation("my/staging/location");
+    try {
+      DataflowPipelineRunner.fromOptions(options);
+      fail("fromOptions should have failed");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+    }
+  }
+
+  @Test
+  public void testNonExistentTempLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setGcsUtil(mockGcsUtil);
+    options.setTempLocation("gs://non-existent-bucket/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString(
+        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+    DataflowPipelineRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNonExistentStagingLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setGcsUtil(mockGcsUtil);
+    options.setStagingLocation("gs://non-existent-bucket/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString(
+        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+    DataflowPipelineRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNoProjectFails() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+
+    options.setRunner(DataflowPipelineRunner.class);
+    // Explicitly set to null to prevent the default instance factory from reading credentials
+    // from a user's environment, causing this test to fail.
+    options.setProject(null);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project id");
+    thrown.expectMessage("when running a Dataflow in the cloud");
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectId() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("foo-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectPrefix() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("google.com:some-project-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectNumber() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project ID");
+    thrown.expectMessage("project number");
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectDescription() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("some project");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project ID");
+    thrown.expectMessage("project description");
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("foo-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Number of worker harness threads");
+    thrown.expectMessage("Please make sure the value is non-negative.");
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testNoStagingLocationAndNoTempLocationFails() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setProject("foo-project");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setGcpCredential(new TestCredential());
+    options.setProject("foo-project");
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setGcpCredential(new TestCredential());
+    options.setProject("foo-project");
+    options.setTempLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    DataflowPipelineRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testInvalidJobName() throws IOException {
+    List<String> invalidNames = Arrays.asList(
+        "invalid_name",
+        "0invalid",
+        "invalid-");
+    List<String> expectedReason = Arrays.asList(
+        "JobName invalid",
+        "JobName invalid",
+        "JobName invalid");
+
+    for (int i = 0; i < invalidNames.size(); ++i) {
+      DataflowPipelineOptions options = buildPipelineOptions();
+      options.setJobName(invalidNames.get(i));
+
+      try {
+        DataflowPipelineRunner.fromOptions(options);
+        fail("Expected IllegalArgumentException for jobName "
+            + options.getJobName());
+      } catch (IllegalArgumentException e) {
+        assertThat(e.getMessage(),
+            containsString(expectedReason.get(i)));
+      }
+    }
+  }
+
+  @Test
+  public void testValidJobName() throws IOException {
+    List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
+        "this-one-is-fairly-long-01234567890123456789");
+
+    for (String name : names) {
+      DataflowPipelineOptions options = buildPipelineOptions();
+      options.setJobName(name);
+
+      DataflowPipelineRunner runner = DataflowPipelineRunner
+          .fromOptions(options);
+      assertNotNull(runner);
+    }
+  }
+
+  /**
+   * A fake PTransform for testing.
+   */
+  public static class TestTransform
+      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
+    public boolean translated = false;
+
+    @Override
+    public PCollection<Integer> apply(PCollection<Integer> input) {
+      return PCollection.<Integer>createPrimitiveOutputInternal(
+          input.getPipeline(),
+          WindowingStrategy.globalDefault(),
+          input.isBounded());
+    }
+
+    @Override
+    protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
+      return input.getCoder();
+    }
+  }
+
+  @Test
+  public void testTransformTranslatorMissing() throws IOException {
+    // Test that we throw if we don't provide a translation.
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    DataflowPipeline p = DataflowPipeline.create(options);
+
+    p.apply(Create.of(Arrays.asList(1, 2, 3)))
+     .apply(new TestTransform());
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(Matchers.containsString("no translator registered"));
+    DataflowPipelineTranslator.fromOptions(options)
+        .translate(p, p.getRunner(), Collections.<DataflowPackage>emptyList());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testTransformTranslator() throws IOException {
+    // Test that we can provide a custom translation
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipeline p = DataflowPipeline.create(options);
+    TestTransform transform = new TestTransform();
+
+    p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
+        .apply(transform);
+
+    DataflowPipelineTranslator translator = DataflowPipelineRunner
+        .fromOptions(options).getTranslator();
+
+    DataflowPipelineTranslator.registerTransformTranslator(
+        TestTransform.class,
+        new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
+          @SuppressWarnings("unchecked")
+          @Override
+          public void translate(
+              TestTransform transform,
+              DataflowPipelineTranslator.TranslationContext context) {
+            transform.translated = true;
+
+            // Note: This is about the minimum needed to fake out a
+            // translation. This obviously isn't a real translation.
+            context.addStep(transform, "TestTranslate");
+            context.addOutput("output", context.getOutput(transform));
+          }
+        });
+
+    translator.translate(
+        p, p.getRunner(), Collections.<DataflowPackage>emptyList());
+    assertTrue(transform.translated);
+  }
+
+  /** Records all the composite transforms visited within the Pipeline. */
+  private static class CompositeTransformRecorder implements PipelineVisitor {
+    private List<PTransform<?, ?>> transforms = new ArrayList<>();
+
+    @Override
+    public void enterCompositeTransform(TransformTreeNode node) {
+      if (node.getTransform() != null) {
+        transforms.add(node.getTransform());
+      }
+    }
+
+    @Override
+    public void leaveCompositeTransform(TransformTreeNode node) {
+    }
+
+    @Override
+    public void visitTransform(TransformTreeNode node) {
+    }
+
+    @Override
+    public void visitValue(PValue value, TransformTreeNode producer) {
+    }
+
+    public List<PTransform<?, ?>> getCompositeTransforms() {
+      return transforms;
+    }
+  }
+
+  @Test
+  public void testApplyIsScopedToExactClass() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowPipeline p = DataflowPipeline.create(options);
+
+    Create.TimestampedValues<String> transform =
+        Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
+    p.apply(transform);
+
+    CompositeTransformRecorder recorder = new CompositeTransformRecorder();
+    p.traverseTopologically(recorder);
+
+    assertThat("Expected to have seen CreateTimestamped composite transform.",
+        recorder.getCompositeTransforms(),
+        Matchers.<PTransform<?, ?>>contains(transform));
+  }
+
+  @Test
+  public void testToString() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    assertEquals("DataflowPipelineRunner#TestJobName",
+        DataflowPipelineRunner.fromOptions(options).toString());
+  }
+
+  private static PipelineOptions makeOptions(boolean streaming) {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    options.setStreaming(streaming);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    return options;
+  }
+
+  private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
+      throws Exception {
+    String mode = streaming ? "streaming" : "batch";
+    thrown.expect(UnsupportedOperationException.class);
+    thrown.expectMessage(
+        "The DataflowPipelineRunner in " + mode + " mode does not support " + name);
+
+    Pipeline p = Pipeline.create(makeOptions(streaming));
+    p.apply(source);
+    p.run();
+  }
+
+  @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);
+  }
+
+  private void testUnsupportedSink(
+      PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
+          throws Exception {
+    thrown.expect(UnsupportedOperationException.class);
+    thrown.expectMessage(
+        "The DataflowPipelineRunner in streaming mode does not support " + name);
+
+    Pipeline p = Pipeline.create(makeOptions(streaming));
+    p.apply(Create.of("foo")).apply(sink);
+    p.run();
+  }
+
+  @Test
+  public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
+    testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
+  }
+
+  @Test
+  public void testTextIOSinkUnsupportedInStreaming() throws Exception {
+    testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
+  }
+
+  @Test
+  public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
+    DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
+        DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBatch(ImmutableList.of("a", "b", "c")), contains(
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
+  }
+
+  @Test
+  public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(
+            new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
+                IntervalWindow.getCoder()));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(
+                    windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
+                )),
+            KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(
+                    windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
+                )));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBatch(inputElements), contains(
+        IsmRecord.of(ImmutableList.of(windowA, 0L),
+            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowA, 1L),
+            WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowA, 2L),
+            WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowB, 0L),
+            WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowB, 1L),
+            WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowC, 0L),
+            WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
+  }
+
+  @Test
+  public void testToIsmRecordForMapLikeDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+            outputForSizeTag,
+            outputForEntrySetTag,
+            windowCoder,
+            keyCoder,
+            ismCoder,
+            false /* unique keys */));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                // same window same key as to previous
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+                // same window different key as to previous
+                KV.of(KV.of(2L, windowA),
+                    WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                // different window same key as to previous
+                KV.of(KV.of(2L, windowB),
+                    WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+                // different window and different key as to previous
+                KV.of(KV.of(3L, windowB),
+                    WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                // different shard
+                KV.of(KV.of(4L, windowC),
+                    WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBatch(inputElements), contains(
+        IsmRecord.of(
+            ImmutableList.of(1L, windowA, 0L),
+            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(1L, windowA, 1L),
+            WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(2L, windowA, 0L),
+            WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(2L, windowB, 0L),
+            WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(3L, windowB, 0L),
+            WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(4L, windowC, 0L),
+            WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
+
+    // Verify the number of unique keys per window.
+    assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+            KV.of(windowC, 1L))
+        ));
+
+    // Verify the output for the unique keys.
+    assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 1L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 3L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+            KV.of(windowC, 4L))
+        ));
+  }
+
+  @Test
+  public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+            outputForSizeTag,
+            outputForEntrySetTag,
+            windowCoder,
+            keyCoder,
+            ismCoder,
+            true /* unique keys */));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+
+    Iterable<KV<Integer,
+                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                // same window same key as to previous
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
+
+    try {
+      doFnTester.processBatch(inputElements);
+      fail("Expected UserCodeException");
+    } catch (UserCodeException e) {
+      assertTrue(e.getCause() instanceof IllegalStateException);
+      IllegalStateException rootCause = (IllegalStateException) e.getCause();
+      assertThat(rootCause.getMessage(), containsString("Unique keys are expected but found key"));
+    }
+  }
+
+  @Test
+  public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+        new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
+            windowCoder));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowA, 2L),
+                    KV.of(windowA, 3L),
+                    KV.of(windowB, 7L))),
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowC, 9L))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBatch(inputElements), contains(
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
+        ));
+  }
+
+  @Test
+  public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+        new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
+            keyCoder, windowCoder));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowA, 2L),
+                    // same window as previous
+                    KV.of(windowA, 3L),
+                    // different window as previous
+                    KV.of(windowB, 3L))),
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowC, 3L))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBatch(inputElements), contains(
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
+        ));
+  }
+
+  @Test
+  public void testToMapDoFn() throws Exception {
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+                  IsmRecord<WindowedValue<TransformedMap<Long,
+                                                         WindowedValue<Long>,
+                                                         Long>>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowC, WindowedValue.of(
+                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    List<IsmRecord<WindowedValue<TransformedMap<Long,
+                                                WindowedValue<Long>,
+                                                Long>>>> output =
+                                                doFnTester.processBatch(inputElements);
+    assertEquals(3, output.size());
+    Map<Long, Long> outputMap;
+
+    outputMap = output.get(0).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
+
+    outputMap = output.get(1).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
+
+    outputMap = output.get(2).getValue().getValue();
+    assertEquals(1, outputMap.size());
+    assertEquals(ImmutableMap.of(4L, 41L), outputMap);
+  }
+
+  @Test
+  public void testToMultimapDoFn() throws Exception {
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+                  IsmRecord<WindowedValue<TransformedMap<Long,
+                                                         Iterable<WindowedValue<Long>>,
+                                                         Iterable<Long>>>>> doFnTester =
+        DoFnTester.of(
+            new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowC, WindowedValue.of(
+                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    List<IsmRecord<WindowedValue<TransformedMap<Long,
+                                                Iterable<WindowedValue<Long>>,
+                                                Iterable<Long>>>>> output =
+                                                doFnTester.processBatch(inputElements);
+    assertEquals(3, output.size());
+    Map<Long, Iterable<Long>> outputMap;
+
+    outputMap = output.get(0).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
+    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+
+    outputMap = output.get(1).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+    assertThat(outputMap.get(3L), containsInAnyOrder(31L));
+
+    outputMap = output.get(2).getValue().getValue();
+    assertEquals(1, outputMap.size());
+    assertThat(outputMap.get(4L), containsInAnyOrder(41L));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
new file mode 100644
index 0000000..ebf0c9f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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 com.google.cloud.dataflow.sdk.runners;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.util.NoopPathValidator;
+import com.google.cloud.dataflow.sdk.util.TestCredential;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowPipeline}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineTest {
+  @Test
+  public void testToString() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    options.setProject("project-id");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    assertEquals("DataflowPipeline#TestJobName",
+        DataflowPipeline.create(options).toString());
+  }
+}


[58/74] incubator-beam git commit: Initial update of imports

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterable.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterable.java
index eef6670..23f8960 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterable.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterable.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 /**
  * An {@link Iterable} that returns {@link Reiterator} iterators.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterator.java
index 5061384..8624790 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Reiterator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;
 
 import java.util.Iterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/package-info.java
index 7d6b3fb..13b94e9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/package-info.java
@@ -16,4 +16,4 @@
  * limitations under the License.
  */
 /** Defines utilities shared by multiple PipelineRunner implementations. **/
-package com.google.cloud.dataflow.sdk.util.common;
+package org.apache.beam.sdk.util.common;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/StateSampler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/StateSampler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/StateSampler.java
index 85b4434..bc16bdd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/StateSampler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/StateSampler.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.common.worker;
+package org.apache.beam.sdk.util.common.worker;
+
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
 
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/package-info.java
index c43557e..f295419 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/worker/package-info.java
@@ -16,4 +16,4 @@
  * limitations under the License.
  */
 /** Defines utilities used to implement the harness that runs user code. **/
-package com.google.cloud.dataflow.sdk.util.common.worker;
+package org.apache.beam.sdk.util.common.worker;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 9398244..3e4b791 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.gcsfs;
+package org.apache.beam.sdk.util.gcsfs;
 
 import com.google.api.services.storage.model.StorageObject;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
index b444415..c7e09ec 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
@@ -16,4 +16,4 @@
  * limitations under the License.
  */
 /** Defines utilities used to interact with Google Cloud Storage. **/
-package com.google.cloud.dataflow.sdk.util.gcsfs;
+package org.apache.beam.sdk.util.gcsfs;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/package-info.java
index 65f53f0..aab0399 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/package-info.java
@@ -16,4 +16,4 @@
  * limitations under the License.
  */
 /** Defines utilities used by the Dataflow SDK. **/
-package com.google.cloud.dataflow.sdk.util;
+package org.apache.beam.sdk.util;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java
index 38c08bb..8dd1678 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
 
 /**
  * State for a single value that is managed by a {@link CombineFn}. This is an internal extension

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java
index 0a9aaa3..c7e6d13 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 /**
  * State containing a bag values. Items can be added to the bag and the contents read out.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java
index a8805f0..1155262 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
 
 /**
  * State that combines multiple {@code InputT} values using a {@link CombineFn} to produce a single

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java
index 83a7b84..3850f04 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.CombineFnUtil;
-import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals.InMemoryState;
-import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.CombineFnUtil;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryState;
+import org.apache.beam.sdk.util.state.StateTag.StateBinder;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java
index c6f07a5..1d5d432 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java
@@ -15,18 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.CombineFnUtil;
-import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.CombineFnUtil;
+import org.apache.beam.sdk.util.state.StateTag.StateBinder;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java
index 8998917..26af7a3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
index 02017ab..ea8a231 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java
index 816460f..973cb9c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 /**
  * Base interface for all state locations.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateAccessor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateAccessor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateAccessor.java
index b42a69c..f54f784 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateAccessor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateAccessor.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java
index ee07010..1d441a6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
 
 /**
  * Information accessible the state API.
@@ -27,7 +27,7 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView;
 public interface StateContext<W extends BoundedWindow> {
   /**
    * Returns the {@code PipelineOptions} specified with the
-   * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}.
+   * {@link org.apache.beam.sdk.runners.PipelineRunner}.
    */
   public abstract PipelineOptions getPipelineOptions();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
index ef0df45..77bb4eb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.values.PCollectionView;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java
index b8de079..24b7226 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.GroupByKey;
 
 /**
  * {@code StateInternals} describes the functionality a runner needs to provide for the

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/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 ef887c4..b02f01d 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
 import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespace.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespace.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespace.java
index b11ea08..9b27b1d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespace.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespace.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaceForTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaceForTest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaceForTest.java
index e94410f..f1050ae 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaceForTest.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaceForTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import java.io.IOException;
 import java.util.Objects;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java
index 43abe6d..b8de27f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
+
+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.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
 import com.google.common.base.Splitter;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java
index 589f7d7..650ebb0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
+
+import org.apache.beam.sdk.util.state.StateTag.StateBinder;
 
-import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder;
 import com.google.common.base.Supplier;
 import com.google.common.collect.Table;
 import com.google.common.collect.Tables;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java
index fc9cb0b..388b5e1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
 
 import java.io.IOException;
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
index e70778d..1e23ae2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+
 import com.google.common.base.MoreObjects;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java
index 9da9170..b432203 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java
index 6cd73ef..f22e9e0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.util.state;
+package org.apache.beam.sdk.util.state;
 
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
 
 import org.joda.time.Instant;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java
index 69f5b7e..1806178 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java
@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableComparator;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.SerializableComparator;
 import com.google.common.base.MoreObjects;
 
 import java.io.Serializable;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.Objects;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
index c1be99c5..e6a2d1f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO.Read;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO.Read;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import java.util.Collection;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
index 0bc140e..a616f5a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
@@ -15,21 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.PubsubIO;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
 
 /**
  * A {@link PCollection PCollection&lt;T&gt;} is an immutable collection of values of type
@@ -52,7 +52,7 @@ import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
  * assigns pubsub message timestamps to elements, and {@link TextIO.Read} assigns
  * the default value {@link BoundedWindow#TIMESTAMP_MIN_VALUE} to elements. User code can
  * explicitly assign timestamps to elements with
- * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp}.
+ * {@link org.apache.beam.sdk.transforms.DoFn.Context#outputWithTimestamp}.
  *
  * <p>Additionally, a {@link PCollection} has an associated
  * {@link WindowFn} and each element is assigned to a set of windows.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
index b3b4ee0..96af69b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Partition;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Partition;
 import com.google.common.collect.ImmutableList;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
index e78b578..b44499b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
 import com.google.common.collect.ImmutableMap;
 
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
index 26bb320..fb3bfab 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
 
 import java.io.Serializable;
 
@@ -31,8 +31,8 @@ import java.io.Serializable;
  * as a side input to a {@link ParDo} transform.
  *
  * <p>A {@link PCollectionView} should always be the output of a
- * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}. It is the joint responsibility of
- * this transform and each {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to implement
+ * {@link org.apache.beam.sdk.transforms.PTransform}. It is the joint responsibility of
+ * this transform and each {@link org.apache.beam.sdk.runners.PipelineRunner} to implement
  * the view in a runner-specific manner.
  *
  * <p>The most common case is using the {@link View} transforms to prepare a {@link PCollection}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
index 5a52ea3..7c05703 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import java.util.Collection;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
index 8d7a7df..3faf6b9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline;
 
 import java.util.Collection;
 
 /**
  * The interface for things that might be input to a
- * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}.
+ * {@link org.apache.beam.sdk.transforms.PTransform}.
  */
 public interface PInput {
   /**
@@ -48,7 +48,7 @@ public interface PInput {
 
   /**
    * <p>After building, finalizes this {@code PInput} to make it ready for
-   * being used as an input to a {@link com.google.cloud.dataflow.sdk.transforms.PTransform}.
+   * being used as an input to a {@link org.apache.beam.sdk.transforms.PTransform}.
    *
    * <p>Automatically invoked whenever {@code apply()} is invoked on
    * this {@code PInput}, so users do not normally call this explicitly.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
index 789c0ab..01e4b36 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
index 24f46a6..4772c47 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * A {@link POutputValueBase} is the abstract base class of

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
index f7c7b70..08eeb13 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * The interface for values that can be input to and output from {@link PTransform PTransforms}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
index ee2a845..8a92fa3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.StringUtils;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.StringUtils;
 
 import java.util.Collection;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
index 26e6b1a..b60a53e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.InstantCoder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java
index e041248..ea06479 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java
@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
-import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean;
-import static com.google.cloud.dataflow.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addString;
+
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.Multiset;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java
index 7546313..8a4beb3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.transforms.ParDo;
 
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
 import com.google.common.collect.ImmutableList;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
index d30c50b..cea79b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;
 
 import com.google.common.collect.Lists;
 import com.google.common.reflect.Invokable;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java
index 34b6640..0387f0a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.values;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException.ReasonCode;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
+package org.apache.beam.sdk.values;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
 
 /**
  * A {@link TypedPValue TypedPValue&lt;T&gt;} is the abstract base class of things that

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
index f254d16..d9bd48c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/package-info.java
@@ -16,38 +16,38 @@
  * limitations under the License.
  */
 /**
- * Defines {@link com.google.cloud.dataflow.sdk.values.PCollection} and other classes for
- * representing data in a {@link com.google.cloud.dataflow.sdk.Pipeline}.
+ * Defines {@link org.apache.beam.sdk.values.PCollection} and other classes for
+ * representing data in a {@link org.apache.beam.sdk.Pipeline}.
  *
  * <p>In particular, see these collection abstractions:
  *
  * <ul>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.PCollection} - an immutable collection of
+ *   <li>{@link org.apache.beam.sdk.values.PCollection} - an immutable collection of
  *     values of type {@code T} and the main representation for data in Dataflow.</li>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.PCollectionView} - an immutable view of a
- *     {@link com.google.cloud.dataflow.sdk.values.PCollection} that can be accessed as a
- *     side input of a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
- *     {@link com.google.cloud.dataflow.sdk.transforms.PTransform}.</li>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} - a heterogeneous tuple of
- *     {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections}
- *     used in cases where a {@link com.google.cloud.dataflow.sdk.transforms.PTransform} takes
+ *   <li>{@link org.apache.beam.sdk.values.PCollectionView} - an immutable view of a
+ *     {@link org.apache.beam.sdk.values.PCollection} that can be accessed as a
+ *     side input of a {@link org.apache.beam.sdk.transforms.ParDo}
+ *     {@link org.apache.beam.sdk.transforms.PTransform}.</li>
+ *   <li>{@link org.apache.beam.sdk.values.PCollectionTuple} - a heterogeneous tuple of
+ *     {@link org.apache.beam.sdk.values.PCollection PCollections}
+ *     used in cases where a {@link org.apache.beam.sdk.transforms.PTransform} takes
  *     or returns multiple
- *     {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections}.</li>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.PCollectionList} - a homogeneous list of
- *     {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections} used, for example,
- *     as input to {@link com.google.cloud.dataflow.sdk.transforms.Flatten}.</li>
+ *     {@link org.apache.beam.sdk.values.PCollection PCollections}.</li>
+ *   <li>{@link org.apache.beam.sdk.values.PCollectionList} - a homogeneous list of
+ *     {@link org.apache.beam.sdk.values.PCollection PCollections} used, for example,
+ *     as input to {@link org.apache.beam.sdk.transforms.Flatten}.</li>
  * </ul>
  *
  * <p>And these classes for individual values play particular roles in Dataflow:
  *
  * <ul>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.KV} - a key/value pair that is used by
- *     keyed transforms, most notably {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}.
+ *   <li>{@link org.apache.beam.sdk.values.KV} - a key/value pair that is used by
+ *     keyed transforms, most notably {@link org.apache.beam.sdk.transforms.GroupByKey}.
  *     </li>
- *   <li>{@link com.google.cloud.dataflow.sdk.values.TimestampedValue} - a timestamp/value pair
+ *   <li>{@link org.apache.beam.sdk.values.TimestampedValue} - a timestamp/value pair
  *     that is used for windowing and handling out-of-order data in streaming execution.</li>
  * </ul>
  *
  * <p>For further details, see the documentation for each class in this package.
  */
-package com.google.cloud.dataflow.sdk.values;
+package org.apache.beam.sdk.values;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/proto/proto2_coder_test_messages.proto
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/proto/proto2_coder_test_messages.proto b/sdks/java/core/src/main/proto/proto2_coder_test_messages.proto
index 7b583b1..b1abe46 100644
--- a/sdks/java/core/src/main/proto/proto2_coder_test_messages.proto
+++ b/sdks/java/core/src/main/proto/proto2_coder_test_messages.proto
@@ -24,7 +24,7 @@ syntax = "proto2";
 
 package proto2_coder_test_messages;
 
-option java_package = "com.google.cloud.dataflow.sdk.coders";
+option java_package = "org.apache.beam.sdk.coders";
 
 message MessageA {
   optional string field1 = 1;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties b/sdks/java/core/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties
deleted file mode 100644
index 5b0a720..0000000
--- a/sdks/java/core/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties
+++ /dev/null
@@ -1,5 +0,0 @@
-# SDK source version.
-version=${pom.version}
-
-build.date=${timestamp}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties b/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
new file mode 100644
index 0000000..5b0a720
--- /dev/null
+++ b/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
@@ -0,0 +1,5 @@
+# SDK source version.
+version=${pom.version}
+
+build.date=${timestamp}
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java
index 0a1e011..2d2a373 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;
 
 import com.google.protobuf.ByteString;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index e6a6426..9d38c1c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.instanceOf;
@@ -24,28 +24,29 @@ import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.testing.ExpectedLogs;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+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.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.TupleTag;
+
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Assert;
@@ -123,7 +124,7 @@ public class PipelineTest {
   }
 
   @Test
-  @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class)
+  @Category(org.apache.beam.sdk.testing.RunnableOnService.class)
   public void testMultipleApply() {
     PTransform<PCollection<? extends String>, PCollection<String>> myTransform =
         addSuffix("+");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
index f6feaaa..35709ed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;
 
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.values.KV;
 
 import org.hamcrest.CoreMatchers;
 import org.hamcrest.Description;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
index ad6a02f..b47c32c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk;
+package org.apache.beam.sdk;
 
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
 
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
index d938a1c..4da8772 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
@@ -23,18 +23,18 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.testing.PAssert;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.SerializableUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.PCollection;
 
 import org.apache.avro.AvroTypeException;
 import org.apache.avro.Schema;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
index d7868d9..f4c22bb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -59,7 +59,7 @@ public class BigEndianIntegerCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "____9Q",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
index 654d5ed..cf148b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -61,7 +61,7 @@ public class BigEndianLongCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "__________U",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
index b9e76a7..1215940 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.common.CounterTestUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -119,7 +119,7 @@ public class ByteArrayCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "CgsM",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
index cdce00c..0494fe0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -63,7 +63,7 @@ public class ByteCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AQ",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java
index 0523030..d65772c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+
 import com.google.common.collect.ImmutableList;
 import com.google.protobuf.ByteString;
 
@@ -59,7 +60,7 @@ public class ByteStringCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java
index bd096dc..a871404 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java
index 9b7ccd4..44d17d4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.instanceOf;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptor;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
index 22fcebb..df440f4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
@@ -15,26 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.CoderRegistry.IncompatibleCoderException;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CoderRegistry.IncompatibleCoderException;
+import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+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.util.CloudObject;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
 import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA;
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-import com.google.cloud.dataflow.sdk.testing.TestPipeline;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
 import com.google.common.collect.ImmutableList;
 import com.google.protobuf.Duration;
 
@@ -387,7 +388,7 @@ public class CoderRegistryTest {
     thrown.expectMessage(allOf(
         containsString("TestGenericT"),
         containsString("erasure"),
-        containsString("com.google.cloud.dataflow.sdk.coders.CoderRegistryTest$TestGenericClass")));
+        containsString("org.apache.beam.sdk.coders.CoderRegistryTest$TestGenericClass")));
     registry.getDefaultCoder(TypeDescriptor.of(
         TestGenericClass.class.getTypeParameters()[0]));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
index dea2fa8..b6378fc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsString;
@@ -23,8 +23,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertThat;
 
-import com.google.cloud.dataflow.sdk.coders.Coder.Context;
-import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
 
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b9724454/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
index 1766e53..a2bc20d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.google.cloud.dataflow.sdk.coders;
+package org.apache.beam.sdk.coders;
 
-import com.google.cloud.dataflow.sdk.testing.CoderProperties;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -66,7 +66,7 @@ public class CollectionCoderTest {
 
   /**
    * Generated data to check that the wire format has not changed. To regenerate, see
-   * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}.
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
   private static final List<String> TEST_ENCODINGS = Arrays.asList(
       "AAAAAA",



[22/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java
deleted file mode 100644
index df9c0c8..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java
+++ /dev/null
@@ -1,112 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-
-/**
- * A {@link ByteCoder} encodes {@link Byte} values in 1 byte using Java serialization.
- */
-public class ByteCoder extends AtomicCoder<Byte> {
-
-  @JsonCreator
-  public static ByteCoder of() {
-    return INSTANCE;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final ByteCoder INSTANCE = new ByteCoder();
-
-  private ByteCoder() {}
-
-  @Override
-  public void encode(Byte value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null Byte");
-    }
-    outStream.write(value.byteValue());
-  }
-
-  @Override
-  public Byte decode(InputStream inStream, Context context)
-      throws IOException, CoderException {
-    try {
-      // value will be between 0-255, -1 for EOF
-      int value = inStream.read();
-      if (value == -1) {
-        throw new EOFException("EOF encountered decoding 1 byte from input stream");
-      }
-      return (byte) value;
-    } catch (EOFException | UTFDataFormatException exn) {
-      // These exceptions correspond to decoding problems, so change
-      // what kind of exception they're branded as.
-      throw new CoderException(exn);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * {@link ByteCoder} will never throw a {@link Coder.NonDeterministicException}; bytes can always
-   * be encoded deterministically.
-   */
-  @Override
-  public void verifyDeterministic() {}
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. This coder is injective.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true}. {@link ByteCoder#getEncodedElementByteSize} returns a constant.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(Byte value, Context context) {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code 1}, the byte size of a {@link Byte} encoded using Java serialization.
-   */
-  @Override
-  protected long getEncodedElementByteSize(Byte value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot estimate size for unsupported null value");
-    }
-    return 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteStringCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteStringCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteStringCoder.java
deleted file mode 100644
index 557d2bd..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteStringCoder.java
+++ /dev/null
@@ -1,107 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.common.io.ByteStreams;
-import com.google.protobuf.ByteString;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * A {@link Coder} for {@link ByteString} objects based on their encoded Protocol Buffer form.
- *
- * <p>When this code is used in a nested {@link Coder.Context}, the serialized {@link ByteString}
- * objects are first delimited by their size.
- */
-public class ByteStringCoder extends AtomicCoder<ByteString> {
-
-  @JsonCreator
-  public static ByteStringCoder of() {
-    return INSTANCE;
-  }
-
-  /***************************/
-
-  private static final ByteStringCoder INSTANCE = new ByteStringCoder();
-
-  private ByteStringCoder() {}
-
-  @Override
-  public void encode(ByteString value, OutputStream outStream, Context context)
-      throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null ByteString");
-    }
-
-    if (!context.isWholeStream) {
-      // ByteString is not delimited, so write its size before its contents.
-      VarInt.encode(value.size(), outStream);
-    }
-    value.writeTo(outStream);
-  }
-
-  @Override
-  public ByteString decode(InputStream inStream, Context context) throws IOException {
-    if (context.isWholeStream) {
-      return ByteString.readFrom(inStream);
-    }
-
-    int size = VarInt.decodeInt(inStream);
-    // ByteString reads to the end of the input stream, so give it a limited stream of exactly
-    // the right length. Also set its chunk size so that the ByteString will contain exactly
-    // one chunk.
-    return ByteString.readFrom(ByteStreams.limit(inStream, size), size);
-  }
-
-  @Override
-  protected long getEncodedElementByteSize(ByteString value, Context context) throws Exception {
-    int size = value.size();
-
-    if (context.isWholeStream) {
-      return size;
-    }
-    return VarInt.getLength(size) + size;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>Returns true; the encoded output of two invocations of {@link ByteStringCoder} in the same
-   * {@link Coder.Context} will be identical if and only if the original {@link ByteString} objects
-   * are equal according to {@link Object#equals}.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return true;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>Returns true. {@link ByteString#size} returns the size of an array and a {@link VarInt}.
-   */
-  @Override
-  public boolean isRegisterByteSizeObserverCheap(ByteString value, Context context) {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CannotProvideCoderException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CannotProvideCoderException.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CannotProvideCoderException.java
deleted file mode 100644
index 66efefa..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CannotProvideCoderException.java
+++ /dev/null
@@ -1,96 +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 com.google.cloud.dataflow.sdk.coders;
-
-/**
- * The exception thrown when a {@link CoderProvider} cannot
- * provide a {@link Coder} that has been requested.
- */
-public class CannotProvideCoderException extends Exception {
-  private final ReasonCode reason;
-
-  public CannotProvideCoderException(String message) {
-    this(message, ReasonCode.UNKNOWN);
-  }
-
-  public CannotProvideCoderException(String message, ReasonCode reason) {
-    super(message);
-    this.reason = reason;
-  }
-
-  public CannotProvideCoderException(String message, Throwable cause) {
-    this(message, cause, ReasonCode.UNKNOWN);
-  }
-
-  public CannotProvideCoderException(String message, Throwable cause, ReasonCode reason) {
-    super(message, cause);
-    this.reason = reason;
-  }
-
-  public CannotProvideCoderException(Throwable cause) {
-    this(cause, ReasonCode.UNKNOWN);
-  }
-
-  public CannotProvideCoderException(Throwable cause, ReasonCode reason) {
-    super(cause);
-    this.reason = reason;
-  }
-
-  /**
-   * @return the reason that Coder inference failed.
-   */
-  public ReasonCode getReason() {
-    return reason;
-  }
-
-  /**
-   * Returns the inner-most {@link CannotProvideCoderException} when they are deeply nested.
-   *
-   * <p>For example, if a coder for {@code List<KV<Integer, Whatsit>>} cannot be provided because
-   * there is no known coder for {@code Whatsit}, the root cause of the exception should be a
-   * CannotProvideCoderException with details pertinent to {@code Whatsit}, suppressing the
-   * intermediate layers.
-   */
-  public Throwable getRootCause() {
-    Throwable cause = getCause();
-    if (cause == null) {
-      return this;
-    } else if (!(cause instanceof CannotProvideCoderException)) {
-      return cause;
-    } else {
-      return ((CannotProvideCoderException) cause).getRootCause();
-    }
-  }
-
-  /**
-   * Indicates the reason that {@link Coder} inference failed.
-   */
-  public static enum ReasonCode {
-    /**
-     * The reason a coder could not be provided is unknown or does have an established
-     * {@link ReasonCode}.
-     */
-    UNKNOWN,
-
-    /**
-     * The reason a coder could not be provided is type erasure, for example when requesting
-     * coder inference for a {@code List<T>} where {@code T} is unknown.
-     */
-    TYPE_ERASURE
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java
deleted file mode 100644
index ff2e10e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java
+++ /dev/null
@@ -1,299 +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 com.google.cloud.dataflow.sdk.coders;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
-import com.google.cloud.dataflow.sdk.util.CloudObject;
-import com.google.cloud.dataflow.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;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Coder Coder&lt;T&gt;} defines how to encode and decode values of type {@code T} into
- * byte streams.
- *
- * <p>{@link Coder} instances are serialized during job creation and deserialized
- * before use, via JSON serialization. See {@link SerializableCoder} for an example of a
- * {@link Coder} that adds a custom field to
- * the {@link Coder} serialization. It provides a constructor annotated with
- * {@link com.fasterxml.jackson.annotation.JsonCreator}, which is a factory method used when
- * deserializing a {@link Coder} instance.
- *
- * <p>{@link Coder} classes for compound types are often composed from coder classes for types
- * contains therein. The composition of {@link Coder} instances into a coder for the compound
- * class is the subject of the {@link CoderFactory} type, which enables automatic generic
- * composition of {@link Coder} classes within the {@link CoderRegistry}. With particular
- * static methods on a compound {@link Coder} class, a {@link CoderFactory} can be automatically
- * inferred. See {@link KvCoder} for an example of a simple compound {@link Coder} that supports
- * automatic composition in the {@link CoderRegistry}.
- *
- * <p>The binary format of a {@link Coder} is identified by {@link #getEncodingId()}; be sure to
- * understand the requirements for evolving coder formats.
- *
- * <p>All methods of a {@link Coder} are required to be thread safe.
- *
- * @param <T> the type of the values being transcoded
- */
-public interface Coder<T> extends Serializable {
-  /** The context in which encoding or decoding is being done. */
-  public static class Context {
-    /**
-     * The outer context: the value being encoded or decoded takes
-     * up the remainder of the record/stream contents.
-     */
-    public static final Context OUTER = new Context(true);
-
-    /**
-     * The nested context: the value being encoded or decoded is
-     * (potentially) a part of a larger record/stream contents, and
-     * may have other parts encoded or decoded after it.
-     */
-    public static final Context NESTED = new Context(false);
-
-    /**
-     * Whether the encoded or decoded value fills the remainder of the
-     * output or input (resp.) record/stream contents.  If so, then
-     * the size of the decoded value can be determined from the
-     * remaining size of the record/stream contents, and so explicit
-     * lengths aren't required.
-     */
-    public final boolean isWholeStream;
-
-    public Context(boolean isWholeStream) {
-      this.isWholeStream = isWholeStream;
-    }
-
-    public Context nested() {
-      return NESTED;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (!(obj instanceof Context)) {
-        return false;
-      }
-      return Objects.equal(isWholeStream, ((Context) obj).isWholeStream);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(isWholeStream);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(Context.class)
-          .addValue(isWholeStream ? "OUTER" : "NESTED").toString();
-    }
-  }
-
-  /**
-   * Encodes the given value of type {@code T} onto the given output stream
-   * in the given context.
-   *
-   * @throws IOException if writing to the {@code OutputStream} fails
-   * for some reason
-   * @throws CoderException if the value could not be encoded for some reason
-   */
-  public void encode(T value, OutputStream outStream, Context context)
-      throws CoderException, IOException;
-
-  /**
-   * Decodes a value of type {@code T} from the given input stream in
-   * the given context.  Returns the decoded value.
-   *
-   * @throws IOException if reading from the {@code InputStream} fails
-   * for some reason
-   * @throws CoderException if the value could not be decoded for some reason
-   */
-  public T decode(InputStream inStream, Context context)
-      throws CoderException, IOException;
-
-  /**
-   * If this is a {@code Coder} for a parameterized type, returns the
-   * list of {@code Coder}s being used for each of the parameters, or
-   * returns {@code null} if this cannot be done or this is not a
-   * parameterized type.
-   */
-  public List<? extends Coder<?>> getCoderArguments();
-
-  /**
-   * Returns the {@link CloudObject} that represents this {@code Coder}.
-   */
-  public CloudObject asCloudObject();
-
-  /**
-   * Throw {@link NonDeterministicException} if the coding is not deterministic.
-   *
-   * <p>In order for a {@code Coder} to be considered deterministic,
-   * the following must be true:
-   * <ul>
-   *   <li>two values that compare as equal (via {@code Object.equals()}
-   *       or {@code Comparable.compareTo()}, if supported) have the same
-   *       encoding.
-   *   <li>the {@code Coder} always produces a canonical encoding, which is the
-   *       same for an instance of an object even if produced on different
-   *       computers at different times.
-   * </ul>
-   *
-   * @throws Coder.NonDeterministicException if this coder is not deterministic.
-   */
-  public void verifyDeterministic() throws Coder.NonDeterministicException;
-
-  /**
-   * Returns {@code true} if this {@link Coder} is injective with respect to {@link Objects#equals}.
-   *
-   * <p>Whenever the encoded bytes of two values are equal, then the original values are equal
-   * according to {@code Objects.equals()}. Note that this is well-defined for {@code null}.
-   *
-   * <p>This condition is most notably false for arrays. More generally, this condition is false
-   * whenever {@code equals()} compares object identity, rather than performing a
-   * semantic/structural comparison.
-   */
-  public boolean consistentWithEquals();
-
-  /**
-   * Returns an object with an {@code Object.equals()} method that represents structural equality
-   * on the argument.
-   *
-   * <p>For any two values {@code x} and {@code y} of type {@code T}, if their encoded bytes are the
-   * same, then it must be the case that {@code structuralValue(x).equals(@code structuralValue(y)}.
-   *
-   * <p>Most notably:
-   * <ul>
-   *   <li>The structural value for an array coder should perform a structural comparison of the
-   *   contents of the arrays, rather than the default behavior of comparing according to object
-   *   identity.
-   *   <li>The structural value for a coder accepting {@code null} should be a proper object with
-   *   an {@code equals()} method, even if the input value is {@code null}.
-   * </ul>
-   *
-   * <p>See also {@link #consistentWithEquals()}.
-   */
-  public Object structuralValue(T value) throws Exception;
-
-  /**
-   * Returns whether {@link #registerByteSizeObserver} cheap enough to
-   * call for every element, that is, if this {@code Coder} can
-   * calculate the byte size of the element to be coded in roughly
-   * constant time (or lazily).
-   *
-   * <p>Not intended to be called by user code, but instead by
-   * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
-   * implementations.
-   */
-  public boolean isRegisterByteSizeObserverCheap(T value, Context context);
-
-  /**
-   * Notifies the {@code ElementByteSizeObserver} about the byte size
-   * of the encoded value using this {@code Coder}.
-   *
-   * <p>Not intended to be called by user code, but instead by
-   * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}
-   * implementations.
-   */
-  public void registerByteSizeObserver(
-      T value, ElementByteSizeObserver observer, Context context)
-      throws Exception;
-
-  /**
-   * An identifier for the binary format written by {@link #encode}.
-   *
-   * <p>This value, along with the fully qualified class name, forms an identifier for the
-   * binary format of this coder. Whenever this value changes, the new encoding is considered
-   * incompatible with the prior format: It is presumed that the prior version of the coder will
-   * be unable to correctly read the new format and the new version of the coder will be unable to
-   * correctly read the old format.
-   *
-   * <p>If the format is changed in a backwards-compatible way (the Coder can still accept data from
-   * the prior format), such as by adding optional fields to a Protocol Buffer or Avro definition,
-   * and you want Dataflow to understand that the new coder is compatible with the prior coder,
-   * this value must remain unchanged. It is then the responsibility of {@link #decode} to correctly
-   * read data from the prior format.
-   */
-  @Experimental(Kind.CODER_ENCODING_ID)
-  public String getEncodingId();
-
-  /**
-   * A collection of encodings supported by {@link #decode} in addition to the encoding
-   * from {@link #getEncodingId()} (which is assumed supported).
-   *
-   * <p><i>This information is not currently used for any purpose</i>. It is descriptive only,
-   * and this method is subject to change.
-   *
-   * @see #getEncodingId()
-   */
-  @Experimental(Kind.CODER_ENCODING_ID)
-  public Collection<String> getAllowedEncodings();
-
-  /**
-   * Exception thrown by {@link Coder#verifyDeterministic()} if the encoding is
-   * not deterministic, including details of why the encoding is not deterministic.
-   */
-  public static class NonDeterministicException extends Throwable {
-    private Coder<?> coder;
-    private List<String> reasons;
-
-    public NonDeterministicException(
-        Coder<?> coder, String reason, @Nullable NonDeterministicException e) {
-      this(coder, Arrays.asList(reason), e);
-    }
-
-    public NonDeterministicException(Coder<?> coder, String reason) {
-      this(coder, Arrays.asList(reason), null);
-    }
-
-    public NonDeterministicException(Coder<?> coder, List<String> reasons) {
-      this(coder, reasons, null);
-    }
-
-    public NonDeterministicException(
-        Coder<?> coder,
-        List<String> reasons,
-        @Nullable NonDeterministicException cause) {
-      super(cause);
-      Preconditions.checkArgument(reasons.size() > 0,
-          "Reasons must not be empty.");
-      this.reasons = reasons;
-      this.coder = coder;
-    }
-
-    public Iterable<String> getReasons() {
-      return reasons;
-    }
-
-    @Override
-    public String getMessage() {
-      return String.format("%s is not deterministic because:\n  %s",
-          coder, Joiner.on("\n  ").join(reasons));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java
deleted file mode 100644
index f7bbb69..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java
+++ /dev/null
@@ -1,37 +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 com.google.cloud.dataflow.sdk.coders;
-
-import java.io.IOException;
-
-/**
- * An {@link Exception} thrown if there is a problem encoding or decoding a value.
- */
-public class CoderException extends IOException {
-  public CoderException(String message) {
-    super(message);
-  }
-
-  public CoderException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public CoderException(Throwable cause) {
-    super(cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactories.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactories.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactories.java
deleted file mode 100644
index 7f788c7..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactories.java
+++ /dev/null
@@ -1,275 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.ParameterizedType;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Static utility methods for creating and working with {@link Coder}s.
- */
-public final class CoderFactories {
-  private CoderFactories() { } // Static utility class
-
-  /**
-   * Creates a {@link CoderFactory} built from particular static methods of a class that
-   * implements {@link Coder}.
-   *
-   * <p>The class must have the following static methods:
-   *
-   * <ul>
-   * <li> {@code
-   * public static Coder<T> of(Coder<X> argCoder1, Coder<Y> argCoder2, ...)
-   * }
-   * <li> {@code
-   * public static List<Object> getInstanceComponents(T exampleValue);
-   * }
-   * </ul>
-   *
-   * <p>The {@code of(...)} method will be used to construct a
-   * {@code Coder<T>} from component {@link Coder}s.
-   * It must accept one {@link Coder} argument for each
-   * generic type parameter of {@code T}. If {@code T} takes no generic
-   * type parameters, then the {@code of()} factory method should take
-   * no arguments.
-   *
-   * <p>The {@code getInstanceComponents} method will be used to
-   * decompose a value during the {@link Coder} inference process,
-   * to automatically choose coders for the components.
-   *
-   * <p>Note that the class {@code T} to be coded may be a
-   * not-yet-specialized generic class.
-   * For a generic class {@code MyClass<X>} and an actual type parameter
-   * {@code Foo}, the {@link CoderFactoryFromStaticMethods} will
-   * accept any {@code Coder<Foo>} and produce a {@code Coder<MyClass<Foo>>}.
-   *
-   * <p>For example, the {@link CoderFactory} returned by
-   * {@code fromStaticMethods(ListCoder.class)}
-   * will produce a {@code Coder<List<X>>} for any {@code Coder Coder<X>}.
-   */
-  public static <T> CoderFactory fromStaticMethods(Class<T> clazz) {
-    return new CoderFactoryFromStaticMethods(clazz);
-  }
-
-  /**
-   * Creates a {@link CoderFactory} that always returns the
-   * given coder.
-   *
-   * <p>The {@code getInstanceComponents} method of this
-   * {@link CoderFactory} always returns an empty list.
-   */
-  public static <T> CoderFactory forCoder(Coder<T> coder) {
-    return new CoderFactoryForCoder<>(coder);
-  }
-
-  /**
-   * See {@link #fromStaticMethods} for a detailed description
-   * of the characteristics of this {@link CoderFactory}.
-   */
-  private static class CoderFactoryFromStaticMethods implements CoderFactory {
-
-    @Override
-    @SuppressWarnings("rawtypes")
-    public Coder<?> create(List<? extends Coder<?>> componentCoders) {
-      try {
-        return (Coder) factoryMethod.invoke(
-            null /* static */, componentCoders.toArray());
-      } catch (IllegalAccessException |
-               IllegalArgumentException |
-               InvocationTargetException |
-               NullPointerException |
-               ExceptionInInitializerError exn) {
-        throw new IllegalStateException(
-            "error when invoking Coder factory method " + factoryMethod,
-            exn);
-      }
-    }
-
-    @Override
-    public List<Object> getInstanceComponents(Object value) {
-      try {
-        @SuppressWarnings("unchecked")
-        List<Object> components =  (List<Object>) getComponentsMethod.invoke(
-            null /* static */, value);
-        return components;
-      } catch (IllegalAccessException
-          | IllegalArgumentException
-          | InvocationTargetException
-          | NullPointerException
-          | ExceptionInInitializerError exn) {
-        throw new IllegalStateException(
-            "error when invoking Coder getComponents method " + getComponentsMethod,
-            exn);
-      }
-    }
-
-    ////////////////////////////////////////////////////////////////////////////////
-
-    // Method to create a coder given component coders
-    // For a Coder class of kind * -> * -> ... n times ... -> *
-    // this has type Coder<?> -> Coder<?> -> ... n times ... -> Coder<T>
-    private Method factoryMethod;
-
-    // Method to decompose a value of type T into its parts.
-    // For a Coder class of kind * -> * -> ... n times ... -> *
-    // this has type T -> List<Object>
-    // where the list has n elements.
-    private Method getComponentsMethod;
-
-    /**
-     * Returns a CoderFactory that invokes the given static factory method
-     * to create the Coder.
-     */
-    private CoderFactoryFromStaticMethods(Class<?> coderClazz) {
-      this.factoryMethod = getFactoryMethod(coderClazz);
-      this.getComponentsMethod = getInstanceComponentsMethod(coderClazz);
-    }
-
-    /**
-     * Returns the static {@code of} constructor method on {@code coderClazz}
-     * if it exists. It is assumed to have one {@link Coder} parameter for
-     * each type parameter of {@code coderClazz}.
-     */
-    private Method getFactoryMethod(Class<?> coderClazz) {
-      Method factoryMethodCandidate;
-
-      // Find the static factory method of coderClazz named 'of' with
-      // the appropriate number of type parameters.
-      int numTypeParameters = coderClazz.getTypeParameters().length;
-      Class<?>[] factoryMethodArgTypes = new Class<?>[numTypeParameters];
-      Arrays.fill(factoryMethodArgTypes, Coder.class);
-      try {
-        factoryMethodCandidate =
-            coderClazz.getDeclaredMethod("of", factoryMethodArgTypes);
-      } catch (NoSuchMethodException | SecurityException exn) {
-        throw new IllegalArgumentException(
-            "cannot register Coder " + coderClazz + ": "
-            + "does not have an accessible method named 'of' with "
-            + numTypeParameters + " arguments of Coder type",
-            exn);
-      }
-      if (!Modifier.isStatic(factoryMethodCandidate.getModifiers())) {
-        throw new IllegalArgumentException(
-            "cannot register Coder " + coderClazz + ": "
-            + "method named 'of' with " + numTypeParameters
-            + " arguments of Coder type is not static");
-      }
-      if (!coderClazz.isAssignableFrom(factoryMethodCandidate.getReturnType())) {
-        throw new IllegalArgumentException(
-            "cannot register Coder " + coderClazz + ": "
-            + "method named 'of' with " + numTypeParameters
-            + " arguments of Coder type does not return a " + coderClazz);
-      }
-      try {
-        if (!factoryMethodCandidate.isAccessible()) {
-          factoryMethodCandidate.setAccessible(true);
-        }
-      } catch (SecurityException exn) {
-        throw new IllegalArgumentException(
-            "cannot register Coder " + coderClazz + ": "
-            + "method named 'of' with " + numTypeParameters
-            + " arguments of Coder type is not accessible",
-            exn);
-      }
-
-      return factoryMethodCandidate;
-    }
-
-    /**
-     * Finds the static method on {@code coderType} to use
-     * to decompose a value of type {@code T} into components,
-     * each corresponding to an argument of the {@code of}
-     * method.
-     */
-    private <T> Method getInstanceComponentsMethod(Class<?> coderClazz) {
-      TypeDescriptor<?> coderType = TypeDescriptor.of(coderClazz);
-      TypeDescriptor<T> argumentType = getCodedType(coderType);
-
-      // getInstanceComponents may be implemented in a superclass,
-      // so we search them all for an applicable method. We do not
-      // try to be clever about finding the best overload. It may
-      // be in a generic superclass, erased to accept an Object.
-      // However, subtypes are listed before supertypes (it is a
-      // topological ordering) so probably the best one will be chosen
-      // if there are more than one (which should be rare)
-      for (TypeDescriptor<?> supertype : coderType.getClasses()) {
-        for (Method method : supertype.getRawType().getDeclaredMethods()) {
-          if (method.getName().equals("getInstanceComponents")) {
-            TypeDescriptor<?> formalArgumentType = supertype.getArgumentTypes(method).get(0);
-            if (formalArgumentType.getRawType().isAssignableFrom(argumentType.getRawType())) {
-              return method;
-            }
-          }
-        }
-      }
-
-      throw new IllegalArgumentException(
-          "cannot create a CoderFactory from " + coderType + ": "
-          + "does not have an accessible method "
-          + "'getInstanceComponents'");
-    }
-
-    /**
-     * If {@code coderType} is a subclass of {@link Coder} for a specific
-     * type {@code T}, returns {@code T.class}. Otherwise, raises IllegalArgumentException.
-     */
-    private <T> TypeDescriptor<T> getCodedType(TypeDescriptor<?> coderType) {
-      for (TypeDescriptor<?> ifaceType : coderType.getInterfaces()) {
-        if (ifaceType.getRawType().equals(Coder.class)) {
-          ParameterizedType coderIface = (ParameterizedType) ifaceType.getType();
-          @SuppressWarnings("unchecked")
-          TypeDescriptor<T> token =
-              (TypeDescriptor<T>) TypeDescriptor.of(coderIface.getActualTypeArguments()[0]);
-          return token;
-        }
-      }
-      throw new IllegalArgumentException(
-          "cannot build CoderFactory from class " + coderType
-          + ": does not implement Coder<T> for any T.");
-    }
-  }
-
-  /**
-   * See {@link #forCoder} for a detailed description of this
-   * {@link CoderFactory}.
-   */
-  private static class CoderFactoryForCoder<T> implements CoderFactory {
-    private Coder<T> coder;
-
-    public CoderFactoryForCoder(Coder<T> coder) {
-      this.coder = coder;
-    }
-
-    @Override
-    public Coder<?> create(List<? extends Coder<?>> componentCoders) {
-      return this.coder;
-    }
-
-    @Override
-    public List<Object> getInstanceComponents(Object value) {
-      return Collections.emptyList();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactory.java
deleted file mode 100644
index 775bfc6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderFactory.java
+++ /dev/null
@@ -1,44 +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 com.google.cloud.dataflow.sdk.coders;
-
-import java.util.List;
-
-/**
- * A {@link CoderFactory} creates coders and decomposes values.
- * It may operate on a parameterized type, such as {@link List},
- * in which case the {@link #create} method accepts a list of
- * coders to use for the type parameters.
- */
-public interface CoderFactory {
-
-  /**
-   * Returns a {@code Coder<?>}, given argument coder to use for
-   * values of a particular type, given the Coders for each of
-   * the type's generic parameter types.
-   */
-  public Coder<?> create(List<? extends Coder<?>> componentCoders);
-
-  /**
-   * Returns a list of objects contained in {@code value}, one per
-   * type argument, or {@code null} if none can be determined.
-   * The list of returned objects should be the same size as the
-   * list of coders required by {@link #create}.
-   */
-  public List<Object> getInstanceComponents(Object value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProvider.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProvider.java
deleted file mode 100644
index f8e29e6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProvider.java
+++ /dev/null
@@ -1,34 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-
-/**
- * A {@link CoderProvider} may create a {@link Coder} for
- * any concrete class.
- */
-public interface CoderProvider {
-
-  /**
-   * Provides a coder for a given class, if possible.
-   *
-   * @throws CannotProvideCoderException if no coder can be provided
-   */
-  public <T> Coder<T> getCoder(TypeDescriptor<T> type) throws CannotProvideCoderException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProviders.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProviders.java
deleted file mode 100644
index 64d258d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderProviders.java
+++ /dev/null
@@ -1,165 +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 com.google.cloud.dataflow.sdk.coders;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.util.InstanceBuilder;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import java.lang.reflect.InvocationTargetException;
-import java.util.List;
-
-/**
- * Static utility methods for working with {@link CoderProvider CoderProviders}.
- */
-public final class CoderProviders {
-
-  // Static utility class
-  private CoderProviders() { }
-
-  /**
-   * Creates a {@link CoderProvider} built from particular static methods of a class that
-   * implements {@link Coder}. The requirements for this method are precisely the requirements
-   * for a {@link Coder} class to be usable with {@link DefaultCoder} annotations.
-   *
-   * <p>The class must have the following static method:
-   *
-   * <pre>{@code
-   * public static Coder<T> of(TypeDescriptor<T> type)
-   * }
-   * </pre>
-   */
-  public static <T> CoderProvider fromStaticMethods(Class<T> clazz) {
-    return new CoderProviderFromStaticMethods(clazz);
-  }
-
-
-  /**
-   * Returns a {@link CoderProvider} that consults each of the provider {@code coderProviders}
-   * and returns the first {@link Coder} provided.
-   *
-   * <p>Note that the order in which the providers are listed matters: While the set of types
-   * handled will be the union of those handled by all of the providers in the list, the actual
-   * {@link Coder} provided by the first successful provider may differ, and may have inferior
-   * properties. For example, not all {@link Coder Coders} are deterministic, handle {@code null}
-   * values, or have comparable performance.
-   */
-  public static CoderProvider firstOf(CoderProvider... coderProviders) {
-    return new FirstOf(ImmutableList.copyOf(coderProviders));
-  }
-
-  ///////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * @see #firstOf
-   */
-  private static class FirstOf implements CoderProvider {
-
-    private Iterable<CoderProvider> providers;
-
-    public FirstOf(Iterable<CoderProvider> providers) {
-      this.providers = providers;
-    }
-
-    @Override
-    public <T> Coder<T> getCoder(TypeDescriptor<T> type) throws CannotProvideCoderException {
-      List<String> messages = Lists.newArrayList();
-      for (CoderProvider provider : providers) {
-        try {
-          return provider.getCoder(type);
-        } catch (CannotProvideCoderException exc) {
-          messages.add(String.format("%s could not provide a Coder for type %s: %s",
-              provider, type, exc.getMessage()));
-        }
-      }
-      throw new CannotProvideCoderException(
-          String.format("Cannot provide coder for type %s: %s.",
-              type, Joiner.on("; ").join(messages)));
-    }
-  }
-
-  private static class CoderProviderFromStaticMethods implements CoderProvider {
-
-    /** If true, then clazz has {@code of(TypeDescriptor)}. If false, {@code of(Class)}. */
-    private final boolean takesTypeDescriptor;
-    private final Class<?> clazz;
-
-    public CoderProviderFromStaticMethods(Class<?> clazz) {
-      // Note that the second condition supports older classes, which only needed to provide
-      // of(Class), not of(TypeDescriptor). Our own classes have updated to accept a
-      // TypeDescriptor. Hence the error message points only to the current specification,
-      // not both acceptable conditions.
-      checkArgument(classTakesTypeDescriptor(clazz) || classTakesClass(clazz),
-          "Class " + clazz.getCanonicalName()
-          + " is missing required static method of(TypeDescriptor).");
-
-      this.takesTypeDescriptor = classTakesTypeDescriptor(clazz);
-      this.clazz = clazz;
-    }
-
-    @Override
-    public <T> Coder<T> getCoder(TypeDescriptor<T> type) throws CannotProvideCoderException {
-      try {
-        if (takesTypeDescriptor) {
-          @SuppressWarnings("unchecked")
-          Coder<T> result = InstanceBuilder.ofType(Coder.class)
-              .fromClass(clazz)
-              .fromFactoryMethod("of")
-              .withArg(TypeDescriptor.class, type)
-              .build();
-          return result;
-        } else {
-          @SuppressWarnings("unchecked")
-          Coder<T> result = InstanceBuilder.ofType(Coder.class)
-              .fromClass(clazz)
-              .fromFactoryMethod("of")
-              .withArg(Class.class, type.getRawType())
-              .build();
-          return result;
-        }
-      } catch (RuntimeException exc) {
-        if (exc.getCause() instanceof InvocationTargetException) {
-          throw new CannotProvideCoderException(exc.getCause().getCause());
-        }
-        throw exc;
-      }
-    }
-
-    private boolean classTakesTypeDescriptor(Class<?> clazz) {
-      try {
-        clazz.getDeclaredMethod("of", TypeDescriptor.class);
-        return true;
-      } catch (NoSuchMethodException | SecurityException exc) {
-        return false;
-      }
-    }
-
-    private boolean classTakesClass(Class<?> clazz) {
-      try {
-        clazz.getDeclaredMethod("of", Class.class);
-        return true;
-      } catch (NoSuchMethodException | SecurityException exc) {
-        return false;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java
deleted file mode 100644
index da9e0a0..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java
+++ /dev/null
@@ -1,844 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException.ReasonCode;
-import com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder;
-import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TimestampedValue;
-import com.google.cloud.dataflow.sdk.values.TypeDescriptor;
-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;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
-import java.lang.reflect.TypeVariable;
-import java.lang.reflect.WildcardType;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link CoderRegistry} allows registering the default {@link Coder} to use for a Java class,
- * and looking up and instantiating the default {@link Coder} for a Java type.
- *
- * <p>{@link CoderRegistry} uses the following mechanisms to determine a default {@link Coder} for a
- * Java class, in order of precedence:
- * <ol>
- *   <li>Registration:
- *     <ul>
- *       <li>A {@link CoderFactory} can be registered to handle a particular class via
- *           {@link #registerCoder(Class, CoderFactory)}.</li>
- *       <li>A {@link Coder} class with the static methods to satisfy
- *           {@link CoderFactories#fromStaticMethods} can be registered via
- *           {@link #registerCoder(Class, Class)}.</li>
- *       <li>Built-in types are registered via
- *           {@link #registerStandardCoders()}.</li>
- *     </ul>
- *   <li>Annotations: {@link DefaultCoder} can be used to annotate a type with
- *       the default {@code Coder} type. The {@link Coder} class must satisfy the requirements
- *       of {@link CoderProviders#fromStaticMethods}.
- *   <li>Fallback: A fallback {@link CoderProvider} is used to attempt to provide a {@link Coder}
- *       for any type. By default, this is {@link SerializableCoder#PROVIDER}, which can provide
- *       a {@link Coder} for any type that is serializable via Java serialization. The fallback
- *       {@link CoderProvider} can be get and set via {@link #getFallbackCoderProvider()}
- *       and {@link #setFallbackCoderProvider}. Multiple fallbacks can be chained together using
- *       {@link CoderProviders#firstOf}.
- * </ol>
- */
-public class CoderRegistry implements CoderProvider {
-
-  private static final Logger LOG = LoggerFactory.getLogger(CoderRegistry.class);
-
-  public CoderRegistry() {
-    setFallbackCoderProvider(
-        CoderProviders.firstOf(ProtoCoder.coderProvider(), SerializableCoder.PROVIDER));
-  }
-
-  /**
-   * Registers standard Coders with this CoderRegistry.
-   */
-  public void registerStandardCoders() {
-    registerCoder(Byte.class, ByteCoder.class);
-    registerCoder(ByteString.class, ByteStringCoder.class);
-    registerCoder(Double.class, DoubleCoder.class);
-    registerCoder(Instant.class, InstantCoder.class);
-    registerCoder(Integer.class, VarIntCoder.class);
-    registerCoder(Iterable.class, IterableCoder.class);
-    registerCoder(KV.class, KvCoder.class);
-    registerCoder(List.class, ListCoder.class);
-    registerCoder(Long.class, VarLongCoder.class);
-    registerCoder(Map.class, MapCoder.class);
-    registerCoder(Set.class, SetCoder.class);
-    registerCoder(String.class, StringUtf8Coder.class);
-    registerCoder(TableRow.class, TableRowJsonCoder.class);
-    registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class);
-    registerCoder(Void.class, VoidCoder.class);
-    registerCoder(byte[].class, ByteArrayCoder.class);
-  }
-
-  /**
-   * Registers {@code coderClazz} as the default {@link Coder} class to handle encoding and
-   * decoding instances of {@code clazz}, overriding prior registrations if any exist.
-   *
-   * <p>Supposing {@code T} is the static type corresponding to the {@code clazz}, then
-   * {@code coderClazz} should have a static factory method with the following signature:
-   *
-   * <pre> {@code
-   * public static Coder<T> of(Coder<X> argCoder1, Coder<Y> argCoder2, ...)
-   * } </pre>
-   *
-   * <p>This method will be called to create instances of {@code Coder<T>} for values of type
-   * {@code T}, passing Coders for each of the generic type parameters of {@code T}.  If {@code T}
-   * takes no generic type parameters, then the {@code of()} factory method should have no
-   * arguments.
-   *
-   * <p>If {@code T} is a parameterized type, then it should additionally have a method with the
-   * following signature:
-   *
-   * <pre> {@code
-   * public static List<Object> getInstanceComponents(T exampleValue);
-   * } </pre>
-   *
-   * <p>This method will be called to decompose a value during the {@link Coder} inference process,
-   * to automatically choose {@link Coder Coders} for the components.
-   *
-   * @param clazz the class of objects to be encoded
-   * @param coderClazz a class with static factory methods to provide {@link Coder Coders}
-   */
-  public void registerCoder(Class<?> clazz, Class<?> coderClazz) {
-    registerCoder(clazz, CoderFactories.fromStaticMethods(coderClazz));
-  }
-
-  /**
-   * Registers {@code coderFactory} as the default {@link CoderFactory} to produce {@code Coder}
-   * instances to decode and encode instances of {@code clazz}. This will override prior
-   * registrations if any exist.
-   */
-  public void registerCoder(Class<?> clazz, CoderFactory coderFactory) {
-    coderFactoryMap.put(clazz, coderFactory);
-  }
-
-  /**
-   * Register the provided {@link Coder} for encoding all values of the specified {@code Class}.
-   * This will override prior registrations if any exist.
-   *
-   * <p>Not for use with generic rawtypes. Instead, register a {@link CoderFactory} via
-   * {@link #registerCoder(Class, CoderFactory)} or ensure your {@code Coder} class has the
-   * appropriate static methods and register it directly via {@link #registerCoder(Class, Class)}.
-   */
-  public <T> void registerCoder(Class<T> rawClazz, Coder<T> coder) {
-    Preconditions.checkArgument(
-      rawClazz.getTypeParameters().length == 0,
-      "CoderRegistry.registerCoder(Class<T>, Coder<T>) may not be used "
-      + "with unspecialized generic classes");
-
-    CoderFactory factory = CoderFactories.forCoder(coder);
-    registerCoder(rawClazz, factory);
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given type.
-   *
-   * @throws CannotProvideCoderException if there is no default Coder.
-   */
-  public <T> Coder<T> getDefaultCoder(TypeDescriptor<T> typeDescriptor)
-      throws CannotProvideCoderException {
-    return getDefaultCoder(typeDescriptor, Collections.<Type, Coder<?>>emptyMap());
-  }
-
-  /**
-   * See {@link #getDefaultCoder(TypeDescriptor)}.
-   */
-  @Override
-  public <T> Coder<T> getCoder(TypeDescriptor<T> typeDescriptor)
-      throws CannotProvideCoderException {
-    return getDefaultCoder(typeDescriptor);
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given type, where the given input
-   * type uses the given {@link Coder}.
-   *
-   * @throws CannotProvideCoderException if there is no default Coder.
-   */
-  public <InputT, OutputT> Coder<OutputT> getDefaultCoder(
-      TypeDescriptor<OutputT> typeDescriptor,
-      TypeDescriptor<InputT> inputTypeDescriptor,
-      Coder<InputT> inputCoder)
-      throws CannotProvideCoderException {
-    return getDefaultCoder(
-        typeDescriptor, getTypeToCoderBindings(inputTypeDescriptor.getType(), inputCoder));
-  }
-
-  /**
-   * Returns the {@link Coder} to use on elements produced by this function, given the {@link Coder}
-   * used for its input elements.
-   */
-  public <InputT, OutputT> Coder<OutputT> getDefaultOutputCoder(
-      SerializableFunction<InputT, OutputT> fn, Coder<InputT> inputCoder)
-      throws CannotProvideCoderException {
-
-    ParameterizedType fnType = (ParameterizedType)
-        TypeDescriptor.of(fn.getClass()).getSupertype(SerializableFunction.class).getType();
-
-    return getDefaultCoder(
-        fn.getClass(),
-        SerializableFunction.class,
-        ImmutableMap.of(fnType.getActualTypeArguments()[0], inputCoder),
-        SerializableFunction.class.getTypeParameters()[1]);
-  }
-
-  /**
-   * Returns the {@link Coder} to use for the specified type parameter specialization of the
-   * subclass, given {@link Coder Coders} to use for all other type parameters (if any).
-   *
-   * @throws CannotProvideCoderException if there is no default Coder.
-   */
-  public <T, OutputT> Coder<OutputT> getDefaultCoder(
-      Class<? extends T> subClass,
-      Class<T> baseClass,
-      Map<Type, ? extends Coder<?>> knownCoders,
-      TypeVariable<?> param)
-      throws CannotProvideCoderException {
-
-    Map<Type, Coder<?>> inferredCoders = getDefaultCoders(subClass, baseClass, knownCoders);
-
-    @SuppressWarnings("unchecked")
-    Coder<OutputT> paramCoderOrNull = (Coder<OutputT>) inferredCoders.get(param);
-    if (paramCoderOrNull != null) {
-      return paramCoderOrNull;
-    } else {
-      throw new CannotProvideCoderException(
-          "Cannot infer coder for type parameter " + param.getName());
-    }
-  }
-
-  /**
-   * Returns the {@link Coder} to use for the provided example value, if it can be determined.
-   *
-   * @throws CannotProvideCoderException if there is no default {@link Coder} or
-   * more than one {@link Coder} matches
-   */
-  public <T> Coder<T> getDefaultCoder(T exampleValue) throws CannotProvideCoderException {
-    Class<?> clazz = exampleValue == null ? Void.class : exampleValue.getClass();
-
-    if (clazz.getTypeParameters().length == 0) {
-      // Trust that getDefaultCoder returns a valid
-      // Coder<T> for non-generic clazz.
-      @SuppressWarnings("unchecked")
-      Coder<T> coder = (Coder<T>) getDefaultCoder(clazz);
-      return coder;
-    } else {
-      CoderFactory factory = getDefaultCoderFactory(clazz);
-
-      List<Object> components = factory.getInstanceComponents(exampleValue);
-      if (components == null) {
-        throw new CannotProvideCoderException(String.format(
-            "Cannot provide coder based on value with class %s: The registered CoderFactory with "
-            + "class %s failed to decompose the value, which is required in order to provide "
-            + "Coders for the components.",
-            clazz.getCanonicalName(), factory.getClass().getCanonicalName()));
-      }
-
-      // componentcoders = components.map(this.getDefaultCoder)
-      List<Coder<?>> componentCoders = new ArrayList<>();
-      for (Object component : components) {
-        try {
-          Coder<?> componentCoder = getDefaultCoder(component);
-          componentCoders.add(componentCoder);
-        } catch (CannotProvideCoderException exc) {
-          throw new CannotProvideCoderException(
-              String.format("Cannot provide coder based on value with class %s",
-                  clazz.getCanonicalName()),
-              exc);
-        }
-      }
-
-      // Trust that factory.create maps from valid component Coders
-      // to a valid Coder<T>.
-      @SuppressWarnings("unchecked")
-      Coder<T> coder = (Coder<T>) factory.create(componentCoders);
-      return coder;
-    }
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given class. The following three
-   * sources for a {@link Coder} will be attempted, in order:
-   *
-   * <ol>
-   *   <li>A {@link Coder} class registered explicitly via a call to {@link #registerCoder},
-   *   <li>A {@link DefaultCoder} annotation on the class,
-   *   <li>This registry's fallback {@link CoderProvider}, which may be able to generate a
-   *   {@link Coder} for an arbitrary class.
-   * </ol>
-   *
-   * @throws CannotProvideCoderException if a {@link Coder} cannot be provided
-   */
-  public <T> Coder<T> getDefaultCoder(Class<T> clazz) throws CannotProvideCoderException {
-
-    CannotProvideCoderException factoryException;
-    try {
-      CoderFactory coderFactory = getDefaultCoderFactory(clazz);
-      LOG.debug("Default coder for {} found by factory", clazz);
-      @SuppressWarnings("unchecked")
-      Coder<T> coder = (Coder<T>) coderFactory.create(Collections.<Coder<?>>emptyList());
-      return coder;
-    } catch (CannotProvideCoderException exc) {
-      factoryException = exc;
-    }
-
-    CannotProvideCoderException annotationException;
-    try {
-      return getDefaultCoderFromAnnotation(clazz);
-    } catch (CannotProvideCoderException exc) {
-      annotationException = exc;
-    }
-
-    CannotProvideCoderException fallbackException;
-    if (getFallbackCoderProvider() != null) {
-      try {
-        return getFallbackCoderProvider().getCoder(TypeDescriptor.<T>of(clazz));
-      } catch (CannotProvideCoderException exc) {
-        fallbackException = exc;
-      }
-    } else {
-      fallbackException = new CannotProvideCoderException("no fallback CoderProvider configured");
-    }
-
-    // Build up the error message and list of causes.
-    StringBuilder messageBuilder = new StringBuilder()
-        .append("Unable to provide a default Coder for ").append(clazz.getCanonicalName())
-        .append(". Correct one of the following root causes:");
-
-    messageBuilder
-        .append("\n  Building a Coder using a registered CoderFactory failed: ")
-        .append(factoryException.getMessage());
-
-    messageBuilder
-        .append("\n  Building a Coder from the @DefaultCoder annotation failed: ")
-        .append(annotationException.getMessage());
-
-    messageBuilder
-        .append("\n  Building a Coder from the fallback CoderProvider failed: ")
-        .append(fallbackException.getMessage());
-
-    throw new CannotProvideCoderException(messageBuilder.toString());
-  }
-
-  /**
-   * Sets the fallback {@link CoderProvider} for this registry. If no other method succeeds in
-   * providing a {@code Coder<T>} for a type {@code T}, then the registry will attempt to create
-   * a {@link Coder} using this {@link CoderProvider}.
-   *
-   * <p>By default, this is set to {@link SerializableCoder#PROVIDER}.
-   *
-   * <p>See {@link #getFallbackCoderProvider}.
-   */
-  public void setFallbackCoderProvider(CoderProvider coderProvider) {
-    fallbackCoderProvider = coderProvider;
-  }
-
-  /**
-   * Returns the fallback {@link CoderProvider} for this registry.
-   *
-   * <p>See {@link #setFallbackCoderProvider}.
-   */
-  public CoderProvider getFallbackCoderProvider() {
-    return fallbackCoderProvider;
-  }
-
-  /**
-   * Returns a {@code Map} from each of {@code baseClass}'s type parameters to the {@link Coder} to
-   * use by default for it, in the context of {@code subClass}'s specialization of
-   * {@code baseClass}.
-   *
-   * <p>If no {@link Coder} can be inferred for a particular type parameter, then that type variable
-   * will be absent from the returned {@code Map}.
-   *
-   * <p>For example, if {@code baseClass} is {@code Map.class}, where {@code Map<K, V>} has type
-   * parameters {@code K} and {@code V}, and {@code subClass} extends {@code Map<String, Integer>}
-   * then the result will map the type variable {@code K} to a {@code Coder<String>} and the
-   * type variable {@code V} to a {@code Coder<Integer>}.
-   *
-   * <p>The {@code knownCoders} parameter can be used to provide known {@link Coder Coders} for any
-   * of the parameters; these will be used to infer the others.
-   *
-   * <p>Note that inference is attempted for every type variable. For a type
-   * {@code MyType<One, Two, Three>} inference will be attempted for all of {@code One},
-   * {@code Two}, {@code Three}, even if the requester only wants a {@link Coder} for {@code Two}.
-   *
-   * <p>For this reason {@code getDefaultCoders} (plural) does not throw an exception if a
-   * {@link Coder} for a particular type variable cannot be inferred, but merely omits the entry
-   * from the returned {@code Map}. It is the responsibility of the caller (usually
-   * {@link #getDefaultCoder} to extract the desired coder or throw a
-   * {@link CannotProvideCoderException} when appropriate.
-   *
-   * @param subClass the concrete type whose specializations are being inferred
-   * @param baseClass the base type, a parameterized class
-   * @param knownCoders a map corresponding to the set of known {@link Coder Coders} indexed by
-   * parameter name
-   *
-   * @deprecated this method is not part of the public interface and will be made private
-   */
-  @Deprecated
-  public <T> Map<Type, Coder<?>> getDefaultCoders(
-      Class<? extends T> subClass,
-      Class<T> baseClass,
-      Map<Type, ? extends Coder<?>> knownCoders) {
-    TypeVariable<Class<T>>[] typeParams = baseClass.getTypeParameters();
-    Coder<?>[] knownCodersArray = new Coder<?>[typeParams.length];
-    for (int i = 0; i < typeParams.length; i++) {
-      knownCodersArray[i] = knownCoders.get(typeParams[i]);
-    }
-    Coder<?>[] resultArray = getDefaultCoders(
-      subClass, baseClass, knownCodersArray);
-    Map<Type, Coder<?>> result = new HashMap<>();
-    for (int i = 0; i < typeParams.length; i++) {
-      if (resultArray[i] != null) {
-        result.put(typeParams[i], resultArray[i]);
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Returns an array listing, for each of {@code baseClass}'s type parameters, the {@link Coder} to
-   * use by default for it, in the context of {@code subClass}'s specialization of
-   * {@code baseClass}.
-   *
-   * <p>If a {@link Coder} cannot be inferred for a type variable, its slot in the resulting array
-   * will be {@code null}.
-   *
-   * <p>For example, if {@code baseClass} is {@code Map.class}, where {@code Map<K, V>} has type
-   * parameters {@code K} and {@code V} in that order, and {@code subClass} extends
-   * {@code Map<String, Integer>} then the result will contain a {@code Coder<String>} and a
-   * {@code Coder<Integer>}, in that order.
-   *
-   * <p>The {@code knownCoders} parameter can be used to provide known {@link Coder Coders} for any
-   * of the type parameters. These will be used to infer the others. If non-null, the length of this
-   * array must match the number of type parameters of {@code baseClass}, and simply be filled with
-   * {@code null} values for each type parameters without a known {@link Coder}.
-   *
-   * <p>Note that inference is attempted for every type variable. For a type
-   * {@code MyType<One, Two, Three>} inference will will be attempted for all of {@code One},
-   * {@code Two}, {@code Three}, even if the requester only wants a {@link Coder} for {@code Two}.
-   *
-   * <p>For this reason {@code getDefaultCoders} (plural) does not throw an exception if a
-   * {@link Coder} for a particular type variable cannot be inferred. Instead, it results in a
-   * {@code null} in the array. It is the responsibility of the caller (usually
-   * {@link #getDefaultCoder} to extract the desired coder or throw a
-   * {@link CannotProvideCoderException} when appropriate.
-   *
-   * @param subClass the concrete type whose specializations are being inferred
-   * @param baseClass the base type, a parameterized class
-   * @param knownCoders an array corresponding to the set of base class type parameters. Each entry
-   *        can be either a {@link Coder} (in which case it will be used for inference) or
-   *        {@code null} (in which case it will be inferred). May be {@code null} to indicate the
-   *        entire set of parameters should be inferred.
-   * @throws IllegalArgumentException if baseClass doesn't have type parameters or if the length of
-   *         {@code knownCoders} is not equal to the number of type parameters of {@code baseClass}.
-   */
-  private <T> Coder<?>[] getDefaultCoders(
-      Class<? extends T> subClass,
-      Class<T> baseClass,
-      @Nullable Coder<?>[] knownCoders) {
-    Type type = TypeDescriptor.of(subClass).getSupertype(baseClass).getType();
-    if (!(type instanceof ParameterizedType)) {
-      throw new IllegalArgumentException(type + " is not a ParameterizedType");
-    }
-    ParameterizedType parameterizedType = (ParameterizedType) type;
-    Type[] typeArgs = parameterizedType.getActualTypeArguments();
-    if (knownCoders == null) {
-      knownCoders = new Coder<?>[typeArgs.length];
-    } else if (typeArgs.length != knownCoders.length) {
-      throw new IllegalArgumentException(
-          String.format("Class %s has %d parameters, but %d coders are requested.",
-              baseClass.getCanonicalName(), typeArgs.length, knownCoders.length));
-    }
-
-    Map<Type, Coder<?>> context = new HashMap<>();
-    for (int i = 0; i < knownCoders.length; i++) {
-      if (knownCoders[i] != null) {
-        try {
-          verifyCompatible(knownCoders[i], typeArgs[i]);
-        } catch (IncompatibleCoderException exn) {
-          throw new IllegalArgumentException(
-              String.format("Provided coders for type arguments of %s contain incompatibilities:"
-                  + " Cannot encode elements of type %s with coder %s",
-                  baseClass,
-                  typeArgs[i], knownCoders[i]),
-              exn);
-        }
-        context.putAll(getTypeToCoderBindings(typeArgs[i], knownCoders[i]));
-      }
-    }
-
-    Coder<?>[] result = new Coder<?>[typeArgs.length];
-    for (int i = 0; i < knownCoders.length; i++) {
-      if (knownCoders[i] != null) {
-        result[i] = knownCoders[i];
-      } else {
-        try {
-          result[i] = getDefaultCoder(typeArgs[i], context);
-        } catch (CannotProvideCoderException exc) {
-          result[i] = null;
-        }
-      }
-    }
-    return result;
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Thrown when a {@link Coder} cannot possibly encode a type, yet has been proposed as a
-   * {@link Coder} for that type.
-   */
-  @VisibleForTesting static class IncompatibleCoderException extends RuntimeException {
-    private Coder<?> coder;
-    private Type type;
-
-    public IncompatibleCoderException(String message, Coder<?> coder, Type type) {
-      super(message);
-      this.coder = coder;
-      this.type = type;
-    }
-
-    public IncompatibleCoderException(String message, Coder<?> coder, Type type, Throwable cause) {
-      super(message, cause);
-      this.coder = coder;
-      this.type = type;
-    }
-
-    public Coder<?> getCoder() {
-      return coder;
-    }
-
-    public Type getType() {
-      return type;
-    }
-  }
-
-  /**
-   * Returns {@code true} if the given {@link Coder} can possibly encode elements
-   * of the given type.
-   */
-  @VisibleForTesting static <T, CoderT extends Coder<T>, CandidateT>
-  void verifyCompatible(CoderT coder, Type candidateType) throws IncompatibleCoderException {
-
-    // Various representations of the coder's class
-    @SuppressWarnings("unchecked")
-    Class<CoderT> coderClass = (Class<CoderT>) coder.getClass();
-    TypeDescriptor<CoderT> coderDescriptor = TypeDescriptor.of(coderClass);
-
-    // Various representations of the actual coded type
-    @SuppressWarnings("unchecked")
-    TypeDescriptor<T> codedDescriptor = CoderUtils.getCodedType(coderDescriptor);
-    @SuppressWarnings("unchecked")
-    Class<T> codedClass = (Class<T>) codedDescriptor.getRawType();
-    Type codedType = codedDescriptor.getType();
-
-    // Various representations of the candidate type
-    @SuppressWarnings("unchecked")
-    TypeDescriptor<CandidateT> candidateDescriptor =
-        (TypeDescriptor<CandidateT>) TypeDescriptor.of(candidateType);
-    @SuppressWarnings("unchecked")
-    Class<CandidateT> candidateClass = (Class<CandidateT>) candidateDescriptor.getRawType();
-
-    // If coder has type Coder<T> where the actual value of T is lost
-    // to erasure, then we cannot rule it out.
-    if (candidateType instanceof TypeVariable) {
-      return;
-    }
-
-    // If the raw types are not compatible, we can certainly rule out
-    // coder compatibility
-    if (!codedClass.isAssignableFrom(candidateClass)) {
-      throw new IncompatibleCoderException(
-          String.format("Cannot encode elements of type %s with coder %s because the"
-              + " coded type %s is not assignable from %s",
-              candidateType, coder, codedClass, candidateType),
-          coder, candidateType);
-    }
-    // we have established that this is a covariant upcast... though
-    // coders are invariant, we are just checking one direction
-    @SuppressWarnings("unchecked")
-    TypeDescriptor<T> candidateOkDescriptor = (TypeDescriptor<T>) candidateDescriptor;
-
-    // If the coded type is a parameterized type where any of the actual
-    // type parameters are not compatible, then the whole thing is certainly not
-    // compatible.
-    if ((codedType instanceof ParameterizedType) && !isNullOrEmpty(coder.getCoderArguments())) {
-      ParameterizedType parameterizedSupertype = ((ParameterizedType)
-           candidateOkDescriptor.getSupertype(codedClass).getType());
-      Type[] typeArguments = parameterizedSupertype.getActualTypeArguments();
-      List<? extends Coder<?>> typeArgumentCoders = coder.getCoderArguments();
-      if (typeArguments.length < typeArgumentCoders.size()) {
-        throw new IncompatibleCoderException(
-            String.format("Cannot encode elements of type %s with coder %s:"
-                + " the generic supertype %s has %s type parameters, which is less than the"
-                + " number of coder arguments %s has (%s).",
-                candidateOkDescriptor, coder,
-                parameterizedSupertype, typeArguments.length,
-                coder, typeArgumentCoders.size()),
-            coder, candidateOkDescriptor.getType());
-      }
-      for (int i = 0; i < typeArgumentCoders.size(); i++) {
-        try {
-          verifyCompatible(
-              typeArgumentCoders.get(i),
-              candidateDescriptor.resolveType(typeArguments[i]).getType());
-        } catch (IncompatibleCoderException exn) {
-          throw new IncompatibleCoderException(
-              String.format("Cannot encode elements of type %s with coder %s"
-                  + " because some component coder is incompatible",
-                  candidateType, coder),
-              coder, candidateType, exn);
-        }
-      }
-    }
-  }
-
-  private static boolean isNullOrEmpty(Collection<?> c) {
-    return c == null || c.size() == 0;
-  }
-
-  /**
-   * The map of classes to the CoderFactories to use to create their
-   * default Coders.
-   */
-  private Map<Class<?>, CoderFactory> coderFactoryMap = new HashMap<>();
-
-  /**
-   * A provider of coders for types where no coder is registered.
-   */
-  private CoderProvider fallbackCoderProvider;
-
-  /**
-   * Returns the {@link CoderFactory} to use to create default {@link Coder Coders} for instances of
-   * the given class, or {@code null} if there is no default {@link CoderFactory} registered.
-   */
-  private CoderFactory getDefaultCoderFactory(Class<?> clazz) throws CannotProvideCoderException {
-    CoderFactory coderFactoryOrNull = coderFactoryMap.get(clazz);
-    if (coderFactoryOrNull != null) {
-      return coderFactoryOrNull;
-    } else {
-      throw new CannotProvideCoderException(
-          String.format("Cannot provide coder based on value with class %s: No CoderFactory has "
-              + "been registered for the class.", clazz.getCanonicalName()));
-    }
-  }
-
-  /**
-   * Returns the {@link Coder} returned according to the {@link CoderProvider} from any
-   * {@link DefaultCoder} annotation on the given class.
-   */
-  private <T> Coder<T> getDefaultCoderFromAnnotation(Class<T> clazz)
-      throws CannotProvideCoderException {
-    DefaultCoder defaultAnnotation = clazz.getAnnotation(DefaultCoder.class);
-    if (defaultAnnotation == null) {
-      throw new CannotProvideCoderException(
-          String.format("Class %s does not have a @DefaultCoder annotation.",
-              clazz.getCanonicalName()));
-    }
-
-    LOG.debug("DefaultCoder annotation found for {} with value {}",
-        clazz, defaultAnnotation.value());
-    CoderProvider coderProvider = CoderProviders.fromStaticMethods(defaultAnnotation.value());
-    return coderProvider.getCoder(TypeDescriptor.of(clazz));
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given type,
-   * in a context where the given types use the given coders.
-   *
-   * @throws CannotProvideCoderException if a coder cannot be provided
-   */
-  private <T> Coder<T> getDefaultCoder(
-      TypeDescriptor<T> typeDescriptor,
-      Map<Type, Coder<?>> typeCoderBindings)
-      throws CannotProvideCoderException {
-
-    Coder<?> defaultCoder = getDefaultCoder(typeDescriptor.getType(), typeCoderBindings);
-    LOG.debug("Default coder for {}: {}", typeDescriptor, defaultCoder);
-    @SuppressWarnings("unchecked")
-    Coder<T> result = (Coder<T>) defaultCoder;
-    return result;
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given type,
-   * in a context where the given types use the given coders.
-   *
-   * @throws CannotProvideCoderException if a coder cannot be provided
-   */
-  private Coder<?> getDefaultCoder(Type type, Map<Type, Coder<?>> typeCoderBindings)
-      throws CannotProvideCoderException {
-    Coder<?> coder = typeCoderBindings.get(type);
-    if (coder != null) {
-      return coder;
-    }
-    if (type instanceof Class<?>) {
-      Class<?> clazz = (Class<?>) type;
-      return getDefaultCoder(clazz);
-    } else if (type instanceof ParameterizedType) {
-      return getDefaultCoder((ParameterizedType) type, typeCoderBindings);
-    } else if (type instanceof TypeVariable || type instanceof WildcardType) {
-      // No default coder for an unknown generic type.
-      throw new CannotProvideCoderException(
-          String.format("Cannot provide a coder for type variable %s"
-          + " (declared by %s) because the actual type is unknown due to erasure.",
-          type,
-          ((TypeVariable<?>) type).getGenericDeclaration()),
-          ReasonCode.TYPE_ERASURE);
-    } else {
-      throw new RuntimeException(
-          "Internal error: unexpected kind of Type: " + type);
-    }
-  }
-
-  /**
-   * Returns the {@link Coder} to use by default for values of the given
-   * parameterized type, in a context where the given types use the
-   * given {@link Coder Coders}.
-   *
-   * @throws CannotProvideCoderException if no coder can be provided
-   */
-  private Coder<?> getDefaultCoder(
-      ParameterizedType type,
-      Map<Type, Coder<?>> typeCoderBindings)
-          throws CannotProvideCoderException {
-
-    CannotProvideCoderException factoryException;
-    try {
-      return getDefaultCoderFromFactory(type, typeCoderBindings);
-    } catch (CannotProvideCoderException exc) {
-      factoryException = exc;
-    }
-
-    CannotProvideCoderException annotationException;
-    try {
-      Class<?> rawClazz = (Class<?>) type.getRawType();
-      return getDefaultCoderFromAnnotation(rawClazz);
-    } catch (CannotProvideCoderException exc) {
-      annotationException = exc;
-    }
-
-    // Build up the error message and list of causes.
-    StringBuilder messageBuilder = new StringBuilder()
-        .append("Unable to provide a default Coder for ").append(type)
-        .append(". Correct one of the following root causes:");
-
-    messageBuilder
-        .append("\n  Building a Coder using a registered CoderFactory failed: ")
-        .append(factoryException.getMessage());
-
-    messageBuilder
-        .append("\n  Building a Coder from the @DefaultCoder annotation failed: ")
-        .append(annotationException.getMessage());
-
-    throw new CannotProvideCoderException(messageBuilder.toString());
-  }
-
-  private Coder<?> getDefaultCoderFromFactory(
-      ParameterizedType type,
-      Map<Type, Coder<?>> typeCoderBindings)
-          throws CannotProvideCoderException {
-    Class<?> rawClazz = (Class<?>) type.getRawType();
-    CoderFactory coderFactory = getDefaultCoderFactory(rawClazz);
-    List<Coder<?>> typeArgumentCoders = new ArrayList<>();
-    for (Type typeArgument : type.getActualTypeArguments()) {
-      try {
-        Coder<?> typeArgumentCoder = getDefaultCoder(typeArgument,
-                                                     typeCoderBindings);
-        typeArgumentCoders.add(typeArgumentCoder);
-      } catch (CannotProvideCoderException exc) {
-         throw new CannotProvideCoderException(
-          String.format("Cannot provide coder for parameterized type %s: %s",
-              type,
-              exc.getMessage()),
-          exc);
-      }
-    }
-    return coderFactory.create(typeArgumentCoders);
-  }
-
-  /**
-   * Returns an immutable {@code Map} from each of the type variables
-   * embedded in the given type to the corresponding types
-   * in the given {@link Coder}.
-   */
-  private Map<Type, Coder<?>> getTypeToCoderBindings(Type type, Coder<?> coder) {
-    if (type instanceof TypeVariable || type instanceof Class) {
-      return ImmutableMap.<Type, Coder<?>>of(type, coder);
-    } else if (type instanceof ParameterizedType) {
-      return getTypeToCoderBindings((ParameterizedType) type, coder);
-    } else {
-      return ImmutableMap.of();
-    }
-  }
-
-  /**
-   * Returns an immutable {@code Map} from the type arguments of the parameterized type to their
-   * corresponding {@link Coder Coders}, and so on recursively for their type parameters.
-   *
-   * <p>This method is simply a specialization to break out the most
-   * elaborate case of {@link #getTypeToCoderBindings(Type, Coder)}.
-   */
-  private Map<Type, Coder<?>> getTypeToCoderBindings(ParameterizedType type, Coder<?> coder) {
-    List<Type> typeArguments = Arrays.asList(type.getActualTypeArguments());
-    List<? extends Coder<?>> coderArguments = coder.getCoderArguments();
-
-    if ((coderArguments == null) || (typeArguments.size() != coderArguments.size())) {
-      return ImmutableMap.of();
-    } else {
-      Map<Type, Coder<?>> typeToCoder = Maps.newHashMap();
-
-      typeToCoder.put(type, coder);
-
-      for (int i = 0; i < typeArguments.size(); i++) {
-        Type typeArgument = typeArguments.get(i);
-        Coder<?> coderArgument = coderArguments.get(i);
-        typeToCoder.putAll(getTypeToCoderBindings(typeArgument, coderArgument));
-      }
-
-      return ImmutableMap.<Type, Coder<?>>builder().putAll(typeToCoder).build();
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java
deleted file mode 100644
index 728f761..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java
+++ /dev/null
@@ -1,74 +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 com.google.cloud.dataflow.sdk.coders;
-
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * A {@link CollectionCoder} encodes {@link Collection Collections} in the format
- * of {@link IterableLikeCoder}.
- */
-public class CollectionCoder<T> extends IterableLikeCoder<T, Collection<T>> {
-
-  public static <T> CollectionCoder<T> of(Coder<T> elemCoder) {
-    return new CollectionCoder<>(elemCoder);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // Internal operations below here.
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return the decoded elements directly, since {@link List} is a subtype of
-   * {@link Collection}.
-   */
-  @Override
-  protected final Collection<T> decodeToIterable(List<T> decodedElements) {
-    return decodedElements;
-  }
-
-  @JsonCreator
-  public static CollectionCoder<?> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
-      List<Object> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
-    return of((Coder<?>) components.get(0));
-  }
-
-  /**
-   * Returns the first element in this collection if it is non-empty,
-   * otherwise returns {@code null}.
-   */
-  public static <T> List<Object> getInstanceComponents(
-      Collection<T> exampleValue) {
-    return getInstanceComponentsHelper(exampleValue);
-  }
-
-  protected CollectionCoder(Coder<T> elemCoder) {
-    super(elemCoder, "Collection");
-  }
-}



[06/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InMemoryWatermarkManager.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InMemoryWatermarkManager.java
deleted file mode 100644
index 193d6a4..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InMemoryWatermarkManager.java
+++ /dev/null
@@ -1,1313 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Ordering;
-import com.google.common.collect.SortedMultiset;
-import com.google.common.collect.TreeMultiset;
-
-import org.joda.time.Instant;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.Objects;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of
- * {@link AppliedPTransform AppliedPTransforms} to provide event-time and completion tracking for
- * in-memory execution. {@link InMemoryWatermarkManager} is designed to update and return a
- * consistent view of watermarks in the presence of concurrent updates.
- *
- * <p>An {@link InMemoryWatermarkManager} is provided with the collection of root
- * {@link AppliedPTransform AppliedPTransforms} and a map of {@link PCollection PCollections} to
- * all the {@link AppliedPTransform AppliedPTransforms} that consume them at construction time.
- *
- * <p>Whenever a root {@link AppliedPTransform transform} produces elements, the
- * {@link InMemoryWatermarkManager} is provided with the produced elements and the output watermark
- * of the producing {@link AppliedPTransform transform}. The
- * {@link InMemoryWatermarkManager watermark manager} is responsible for computing the watermarks
- * of all {@link AppliedPTransform transforms} that consume one or more
- * {@link PCollection PCollections}.
- *
- * <p>Whenever a non-root {@link AppliedPTransform} finishes processing one or more in-flight
- * elements (referred to as the input {@link CommittedBundle bundle}), the following occurs
- * atomically:
- * <ul>
- *  <li>All of the in-flight elements are removed from the collection of pending elements for the
- *      {@link AppliedPTransform}.</li>
- *  <li>All of the elements produced by the {@link AppliedPTransform} are added to the collection
- *      of pending elements for each {@link AppliedPTransform} that consumes them.</li>
- *  <li>The input watermark for the {@link AppliedPTransform} becomes the maximum value of
- *    <ul>
- *      <li>the previous input watermark</li>
- *      <li>the minimum of
- *        <ul>
- *          <li>the timestamps of all currently pending elements</li>
- *          <li>all input {@link PCollection} watermarks</li>
- *        </ul>
- *      </li>
- *    </ul>
- *  </li>
- *  <li>The output watermark for the {@link AppliedPTransform} becomes the maximum of
- *    <ul>
- *      <li>the previous output watermark</li>
- *      <li>the minimum of
- *        <ul>
- *          <li>the current input watermark</li>
- *          <li>the current watermark holds</li>
- *        </ul>
- *      </li>
- *    </ul>
- *  </li>
- *  <li>The watermark of the output {@link PCollection} can be advanced to the output watermark of
- *      the {@link AppliedPTransform}</li>
- *  <li>The watermark of all downstream {@link AppliedPTransform AppliedPTransforms} can be
- *      advanced.</li>
- * </ul>
- *
- * <p>The watermark of a {@link PCollection} is equal to the output watermark of the
- * {@link AppliedPTransform} that produces it.
- *
- * <p>The watermarks for a {@link PTransform} are updated as follows when output is committed:<pre>
- * Watermark_In'  = MAX(Watermark_In, MIN(U(TS_Pending), U(Watermark_InputPCollection)))
- * Watermark_Out' = MAX(Watermark_Out, MIN(Watermark_In', U(StateHold)))
- * Watermark_PCollection = Watermark_Out_ProducingPTransform
- * </pre>
- */
-public class InMemoryWatermarkManager {
-  /**
-   * The watermark of some {@link Pipeline} element, usually a {@link PTransform} or a
-   * {@link PCollection}.
-   *
-   * <p>A watermark is a monotonically increasing value, which represents the point up to which the
-   * system believes it has received all of the data. Data that arrives with a timestamp that is
-   * before the watermark is considered late. {@link BoundedWindow#TIMESTAMP_MAX_VALUE} is a special
-   * timestamp which indicates we have received all of the data and there will be no more on-time or
-   * late data. This value is represented by {@link InMemoryWatermarkManager#THE_END_OF_TIME}.
-   */
-  private static interface Watermark {
-    /**
-     * Returns the current value of this watermark.
-     */
-    Instant get();
-
-    /**
-     * Refreshes the value of this watermark from its input watermarks and watermark holds.
-     *
-     * @return true if the value of the watermark has changed (and thus dependent watermark must
-     *         also be updated
-     */
-    WatermarkUpdate refresh();
-  }
-
-  /**
-   * The result of computing a {@link Watermark}.
-   */
-  private static enum WatermarkUpdate {
-    /** The watermark is later than the value at the previous time it was computed. */
-    ADVANCED(true),
-    /** The watermark is equal to the value at the previous time it was computed. */
-    NO_CHANGE(false);
-
-    private final boolean advanced;
-
-    private WatermarkUpdate(boolean advanced) {
-      this.advanced = advanced;
-    }
-
-    public boolean isAdvanced() {
-      return advanced;
-    }
-
-    /**
-     * Returns the {@link WatermarkUpdate} that is a result of combining the two watermark updates.
-     *
-     * If either of the input {@link WatermarkUpdate WatermarkUpdates} were advanced, the result
-     * {@link WatermarkUpdate} has been advanced.
-     */
-    public WatermarkUpdate union(WatermarkUpdate that) {
-      if (this.advanced) {
-        return this;
-      }
-      return that;
-    }
-
-    /**
-     * Returns the {@link WatermarkUpdate} based on the former and current
-     * {@link Instant timestamps}.
-     */
-    public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTime) {
-      if (currentTime.isAfter(oldTime)) {
-        return ADVANCED;
-      }
-      return NO_CHANGE;
-    }
-  }
-
-  /**
-   * The input {@link Watermark} of an {@link AppliedPTransform}.
-   *
-   * <p>At any point, the value of an {@link AppliedPTransformInputWatermark} is equal to the
-   * minimum watermark across all of its input {@link Watermark Watermarks}, and the minimum
-   * timestamp of all of the pending elements, restricted to be monotonically increasing.
-   *
-   * <p>See {@link #refresh()} for more information.
-   */
-  private static class AppliedPTransformInputWatermark implements Watermark {
-    private final Collection<? extends Watermark> inputWatermarks;
-    private final SortedMultiset<WindowedValue<?>> pendingElements;
-    private final Map<Object, NavigableSet<TimerData>> objectTimers;
-
-    private AtomicReference<Instant> currentWatermark;
-
-    public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
-      this.inputWatermarks = inputWatermarks;
-      this.pendingElements = TreeMultiset.create(PENDING_ELEMENT_COMPARATOR);
-      this.objectTimers = new HashMap<>();
-      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    @Override
-    public Instant get() {
-      return currentWatermark.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link AppliedPTransformInputWatermark} becomes
-     * equal to the maximum value of
-     * <ul>
-     *   <li>the previous input watermark</li>
-     *   <li>the minimum of
-     *     <ul>
-     *       <li>the timestamps of all currently pending elements</li>
-     *       <li>all input {@link PCollection} watermarks</li>
-     *     </ul>
-     *   </li>
-     * </ul>
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldWatermark = currentWatermark.get();
-      Instant minInputWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE;
-      for (Watermark inputWatermark : inputWatermarks) {
-        minInputWatermark = INSTANT_ORDERING.min(minInputWatermark, inputWatermark.get());
-      }
-      if (!pendingElements.isEmpty()) {
-        minInputWatermark = INSTANT_ORDERING.min(
-            minInputWatermark, pendingElements.firstEntry().getElement().getTimestamp());
-      }
-      Instant newWatermark = INSTANT_ORDERING.max(oldWatermark, minInputWatermark);
-      currentWatermark.set(newWatermark);
-      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
-    }
-
-    private synchronized void addPendingElements(Iterable<? extends WindowedValue<?>> newPending) {
-      for (WindowedValue<?> pendingElement : newPending) {
-        pendingElements.add(pendingElement);
-      }
-    }
-
-    private synchronized void removePendingElements(
-        Iterable<? extends WindowedValue<?>> finishedElements) {
-      for (WindowedValue<?> finishedElement : finishedElements) {
-        pendingElements.remove(finishedElement);
-      }
-    }
-
-    private synchronized void updateTimers(TimerUpdate update) {
-      NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
-      if (keyTimers == null) {
-        keyTimers = new TreeSet<>();
-        objectTimers.put(update.key, keyTimers);
-      }
-      for (TimerData timer : update.setTimers) {
-        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
-          keyTimers.add(timer);
-        }
-      }
-      for (TimerData timer : update.deletedTimers) {
-        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
-          keyTimers.remove(timer);
-        }
-      }
-      // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
-    }
-
-    private synchronized Map<Object, List<TimerData>> extractFiredEventTimeTimers() {
-      return extractFiredTimers(currentWatermark.get(), objectTimers);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(AppliedPTransformInputWatermark.class)
-          .add("pendingElements", pendingElements)
-          .add("currentWatermark", currentWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * The output {@link Watermark} of an {@link AppliedPTransform}.
-   *
-   * <p>The value of an {@link AppliedPTransformOutputWatermark} is equal to the minimum of the
-   * current watermark hold and the {@link AppliedPTransformInputWatermark} for the same
-   * {@link AppliedPTransform}, restricted to be monotonically increasing. See
-   * {@link #refresh()} for more information.
-   */
-  private static class AppliedPTransformOutputWatermark implements Watermark {
-    private final Watermark inputWatermark;
-    private final PerKeyHolds holds;
-    private AtomicReference<Instant> currentWatermark;
-
-    public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
-      this.inputWatermark = inputWatermark;
-      holds = new PerKeyHolds();
-      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    public synchronized void updateHold(Object key, Instant newHold) {
-      if (newHold == null) {
-        holds.removeHold(key);
-      } else {
-        holds.updateHold(key, newHold);
-      }
-    }
-
-    @Override
-    public Instant get() {
-      return currentWatermark.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link AppliedPTransformOutputWatermark} becomes
-     * equal to the maximum value of:
-     * <ul>
-     *   <li>the previous output watermark</li>
-     *   <li>the minimum of
-     *     <ul>
-     *       <li>the current input watermark</li>
-     *       <li>the current watermark holds</li>
-     *     </ul>
-     *   </li>
-     * </ul>
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldWatermark = currentWatermark.get();
-      Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
-      newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
-      currentWatermark.set(newWatermark);
-      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(AppliedPTransformOutputWatermark.class)
-          .add("holds", holds)
-          .add("currentWatermark", currentWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * The input {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
-   * {@link AppliedPTransform}.
-   *
-   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeInputWatermark} is equal
-   * to the minimum across all pending bundles at the {@link AppliedPTransform} and all upstream
-   * {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the input
-   * synchronized processing time at any step is equal to the maximum of:
-   * <ul>
-   *   <li>The most recently returned synchronized processing input time
-   *   <li>The minimum of
-   *     <ul>
-   *       <li>The current processing time
-   *       <li>The current synchronized processing time input hold
-   *     </ul>
-   * </ul>
-   */
-  private static class SynchronizedProcessingTimeInputWatermark implements Watermark {
-    private final Collection<? extends Watermark> inputWms;
-    private final Collection<CommittedBundle<?>> pendingBundles;
-    private final Map<Object, NavigableSet<TimerData>> processingTimers;
-    private final Map<Object, NavigableSet<TimerData>> synchronizedProcessingTimers;
-
-    private final PriorityQueue<TimerData> pendingTimers;
-
-    private AtomicReference<Instant> earliestHold;
-
-    public SynchronizedProcessingTimeInputWatermark(Collection<? extends Watermark> inputWms) {
-      this.inputWms = inputWms;
-      this.pendingBundles = new HashSet<>();
-      this.processingTimers = new HashMap<>();
-      this.synchronizedProcessingTimers = new HashMap<>();
-      this.pendingTimers = new PriorityQueue<>();
-      Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
-      for (Watermark wm : inputWms) {
-        initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
-      }
-      earliestHold = new AtomicReference<>(initialHold);
-    }
-
-    @Override
-    public Instant get() {
-      return earliestHold.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeInputWatermark}
-     * becomes equal to the minimum value of
-     * <ul>
-     *   <li>the timestamps of all currently pending bundles</li>
-     *   <li>all input {@link PCollection} synchronized processing time watermarks</li>
-     * </ul>
-     *
-     * <p>Note that this value is not monotonic, but the returned value for the synchronized
-     * processing time must be.
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldHold = earliestHold.get();
-      Instant minTime = THE_END_OF_TIME.get();
-      for (Watermark input : inputWms) {
-        minTime = INSTANT_ORDERING.min(minTime, input.get());
-      }
-      for (CommittedBundle<?> bundle : pendingBundles) {
-        // TODO: Track elements in the bundle by the processing time they were output instead of
-        // entire bundles. Requried to support arbitrarily splitting and merging bundles between
-        // steps
-        minTime = INSTANT_ORDERING.min(minTime, bundle.getSynchronizedProcessingOutputWatermark());
-      }
-      earliestHold.set(minTime);
-      return WatermarkUpdate.fromTimestamps(oldHold, minTime);
-    }
-
-    public synchronized void addPending(CommittedBundle<?> bundle) {
-      pendingBundles.add(bundle);
-    }
-
-    public synchronized void removePending(CommittedBundle<?> bundle) {
-      pendingBundles.remove(bundle);
-    }
-
-    /**
-     * Return the earliest timestamp of the earliest timer that has not been completed. This is
-     * either the earliest timestamp across timers that have not been completed, or the earliest
-     * timestamp across timers that have been delivered but have not been completed.
-     */
-    public synchronized Instant getEarliestTimerTimestamp() {
-      Instant earliest = THE_END_OF_TIME.get();
-      for (NavigableSet<TimerData> timers : processingTimers.values()) {
-        if (!timers.isEmpty()) {
-          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
-        }
-      }
-      for (NavigableSet<TimerData> timers : synchronizedProcessingTimers.values()) {
-        if (!timers.isEmpty()) {
-          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
-        }
-      }
-      if (!pendingTimers.isEmpty()) {
-        earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
-      }
-      return earliest;
-    }
-
-    private synchronized void updateTimers(TimerUpdate update) {
-      for (TimerData completedTimer : update.completedTimers) {
-        pendingTimers.remove(completedTimer);
-      }
-      Map<TimeDomain, NavigableSet<TimerData>> timerMap = timerMap(update.key);
-      for (TimerData addedTimer : update.setTimers) {
-        NavigableSet<TimerData> timerQueue = timerMap.get(addedTimer.getDomain());
-        if (timerQueue != null) {
-          timerQueue.add(addedTimer);
-        }
-      }
-      for (TimerData deletedTimer : update.deletedTimers) {
-        NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
-        if (timerQueue != null) {
-          timerQueue.remove(deletedTimer);
-        }
-      }
-    }
-
-    private synchronized Map<Object, List<TimerData>> extractFiredDomainTimers(
-        TimeDomain domain, Instant firingTime) {
-      Map<Object, List<TimerData>> firedTimers;
-      switch (domain) {
-        case PROCESSING_TIME:
-          firedTimers = extractFiredTimers(firingTime, processingTimers);
-          break;
-        case SYNCHRONIZED_PROCESSING_TIME:
-          firedTimers =
-              extractFiredTimers(
-                  INSTANT_ORDERING.min(firingTime, earliestHold.get()),
-                  synchronizedProcessingTimers);
-          break;
-        default:
-          throw new IllegalArgumentException(
-              "Called getFiredTimers on a Synchronized Processing Time watermark"
-                  + " and gave a non-processing time domain "
-                  + domain);
-      }
-      for (Map.Entry<Object, ? extends Collection<TimerData>> firedTimer : firedTimers.entrySet()) {
-        pendingTimers.addAll(firedTimer.getValue());
-      }
-      return firedTimers;
-    }
-
-    private Map<TimeDomain, NavigableSet<TimerData>> timerMap(Object key) {
-      NavigableSet<TimerData> processingQueue = processingTimers.get(key);
-      if (processingQueue == null) {
-        processingQueue = new TreeSet<>();
-        processingTimers.put(key, processingQueue);
-      }
-      NavigableSet<TimerData> synchronizedProcessingQueue =
-          synchronizedProcessingTimers.get(key);
-      if (synchronizedProcessingQueue == null) {
-        synchronizedProcessingQueue = new TreeSet<>();
-        synchronizedProcessingTimers.put(key, synchronizedProcessingQueue);
-      }
-      EnumMap<TimeDomain, NavigableSet<TimerData>> result = new EnumMap<>(TimeDomain.class);
-      result.put(TimeDomain.PROCESSING_TIME, processingQueue);
-      result.put(TimeDomain.SYNCHRONIZED_PROCESSING_TIME, synchronizedProcessingQueue);
-      return result;
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(SynchronizedProcessingTimeInputWatermark.class)
-          .add("earliestHold", earliestHold)
-          .toString();
-    }
-  }
-
-  /**
-   * The output {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
-   * {@link AppliedPTransform}.
-   *
-   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeOutputWatermark} is
-   * equal to the minimum across all incomplete timers at the {@link AppliedPTransform} and all
-   * upstream {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the output
-   * synchronized processing time at any step is equal to the maximum of:
-   * <ul>
-   *   <li>The most recently returned synchronized processing output time
-   *   <li>The minimum of
-   *     <ul>
-   *       <li>The current processing time
-   *       <li>The current synchronized processing time output hold
-   *     </ul>
-   * </ul>
-   */
-  private static class SynchronizedProcessingTimeOutputWatermark implements Watermark {
-    private final SynchronizedProcessingTimeInputWatermark inputWm;
-    private AtomicReference<Instant> latestRefresh;
-
-    public SynchronizedProcessingTimeOutputWatermark(
-        SynchronizedProcessingTimeInputWatermark inputWm) {
-      this.inputWm = inputWm;
-      this.latestRefresh = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    @Override
-    public Instant get() {
-      return latestRefresh.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeOutputWatermark}
-     * becomes equal to the minimum value of:
-     * <ul>
-     *   <li>the current input watermark.
-     *   <li>all {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} timers that are based on the input
-     *       watermark.
-     *   <li>all {@link TimeDomain#PROCESSING_TIME} timers that are based on the input watermark.
-     * </ul>
-     *
-     * <p>Note that this value is not monotonic, but the returned value for the synchronized
-     * processing time must be.
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      // Hold the output synchronized processing time to the input watermark, which takes into
-      // account buffered bundles, and the earliest pending timer, which determines what to hold
-      // downstream timers to.
-      Instant oldRefresh = latestRefresh.get();
-      Instant newTimestamp =
-          INSTANT_ORDERING.min(inputWm.get(), inputWm.getEarliestTimerTimestamp());
-      latestRefresh.set(newTimestamp);
-      return WatermarkUpdate.fromTimestamps(oldRefresh, newTimestamp);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(SynchronizedProcessingTimeOutputWatermark.class)
-          .add("latestRefresh", latestRefresh)
-          .toString();
-    }
-  }
-
-  /**
-   * The {@code Watermark} that is after the latest time it is possible to represent in the global
-   * window. This is a distinguished value representing a complete {@link PTransform}.
-   */
-  private static final Watermark THE_END_OF_TIME = new Watermark() {
-        @Override
-        public WatermarkUpdate refresh() {
-          // THE_END_OF_TIME is a distinguished value that cannot be advanced.
-          return WatermarkUpdate.NO_CHANGE;
-        }
-
-        @Override
-        public Instant get() {
-          return BoundedWindow.TIMESTAMP_MAX_VALUE;
-        }
-      };
-
-  private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
-
-  /**
-   * An ordering that compares windowed values by timestamp, then arbitrarily. This ensures that
-   * {@link WindowedValue WindowedValues} will be sorted by timestamp, while two different
-   * {@link WindowedValue WindowedValues} with the same timestamp are not considered equal.
-   */
-  private static final Ordering<WindowedValue<? extends Object>> PENDING_ELEMENT_COMPARATOR =
-      (new WindowedValueByTimestampComparator()).compound(Ordering.arbitrary());
-
-  /**
-   * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
-   * latestTime argument and put in in the result with the same key, then remove all of the keys
-   * which have no more pending timers.
-   *
-   * The result collection retains ordering of timers (from earliest to latest).
-   */
-  private static Map<Object, List<TimerData>> extractFiredTimers(
-      Instant latestTime, Map<Object, NavigableSet<TimerData>> objectTimers) {
-    Map<Object, List<TimerData>> result = new HashMap<>();
-    Set<Object> emptyKeys = new HashSet<>();
-    for (Map.Entry<Object, NavigableSet<TimerData>> pendingTimers : objectTimers.entrySet()) {
-      NavigableSet<TimerData> timers = pendingTimers.getValue();
-      if (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
-        ArrayList<TimerData> keyFiredTimers = new ArrayList<>();
-        result.put(pendingTimers.getKey(), keyFiredTimers);
-        while (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
-          keyFiredTimers.add(timers.first());
-          timers.remove(timers.first());
-        }
-      }
-      if (timers.isEmpty()) {
-        emptyKeys.add(pendingTimers.getKey());
-      }
-    }
-    objectTimers.keySet().removeAll(emptyKeys);
-    return result;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * The {@link Clock} providing the current time in the {@link TimeDomain#PROCESSING_TIME} domain.
-   */
-  private final Clock clock;
-
-  /**
-   * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
-   * that consume that {@link PCollection}.
-   */
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
-
-  /**
-   * The input and output watermark of each {@link AppliedPTransform}.
-   */
-  private final Map<AppliedPTransform<?, ?, ?>, TransformWatermarks> transformToWatermarks;
-
-  /**
-   * Creates a new {@link InMemoryWatermarkManager}. All watermarks within the newly created
-   * {@link InMemoryWatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
-   * minimum watermark, with no watermark holds or pending elements.
-   *
-   * @param rootTransforms the root-level transforms of the {@link Pipeline}
-   * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
-   *                  transforms that consume it as a part of their input
-   */
-  public static InMemoryWatermarkManager create(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
-    return new InMemoryWatermarkManager(clock, rootTransforms, consumers);
-  }
-
-  private InMemoryWatermarkManager(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
-    this.clock = clock;
-    this.consumers = consumers;
-
-    transformToWatermarks = new HashMap<>();
-
-    for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
-      getTransformWatermark(rootTransform);
-    }
-    for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
-      for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
-        getTransformWatermark(transform);
-      }
-    }
-  }
-
-  private TransformWatermarks getTransformWatermark(AppliedPTransform<?, ?, ?> transform) {
-    TransformWatermarks wms = transformToWatermarks.get(transform);
-    if (wms == null) {
-      List<Watermark> inputCollectionWatermarks = getInputWatermarks(transform);
-      AppliedPTransformInputWatermark inputWatermark =
-          new AppliedPTransformInputWatermark(inputCollectionWatermarks);
-      AppliedPTransformOutputWatermark outputWatermark =
-          new AppliedPTransformOutputWatermark(inputWatermark);
-
-      SynchronizedProcessingTimeInputWatermark inputProcessingWatermark =
-          new SynchronizedProcessingTimeInputWatermark(getInputProcessingWatermarks(transform));
-      SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark =
-          new SynchronizedProcessingTimeOutputWatermark(inputProcessingWatermark);
-
-      wms =
-          new TransformWatermarks(
-              inputWatermark, outputWatermark, inputProcessingWatermark, outputProcessingWatermark);
-      transformToWatermarks.put(transform, wms);
-    }
-    return wms;
-  }
-
-  private Collection<Watermark> getInputProcessingWatermarks(
-      AppliedPTransform<?, ?, ?> transform) {
-    ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
-    if (inputs.isEmpty()) {
-      inputWmsBuilder.add(THE_END_OF_TIME);
-    }
-    for (PValue pvalue : inputs) {
-      Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal())
-              .synchronizedProcessingOutputWatermark;
-      inputWmsBuilder.add(producerOutputWatermark);
-    }
-    return inputWmsBuilder.build();
-  }
-
-  private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
-    if (inputs.isEmpty()) {
-      inputWatermarksBuilder.add(THE_END_OF_TIME);
-    }
-    for (PValue pvalue : inputs) {
-      Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
-      inputWatermarksBuilder.add(producerOutputWatermark);
-    }
-    List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
-    return inputCollectionWatermarks;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Gets the input and output watermarks for an {@link AppliedPTransform}. If the
-   * {@link AppliedPTransform PTransform} has not processed any elements, return a watermark of
-   * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
-   *
-   * @return a snapshot of the input watermark and output watermark for the provided transform
-   */
-  public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    return transformToWatermarks.get(transform);
-  }
-
-  /**
-   * Updates the watermarks of a transform with one or more inputs.
-   *
-   * <p>Each transform has two monotonically increasing watermarks: the input watermark, which can,
-   * at any time, be updated to equal:
-   * <pre>
-   * MAX(CurrentInputWatermark, MIN(PendingElements, InputPCollectionWatermarks))
-   * </pre>
-   * and the output watermark, which can, at any time, be updated to equal:
-   * <pre>
-   * MAX(CurrentOutputWatermark, MIN(InputWatermark, WatermarkHolds))
-   * </pre>.
-   *
-   * @param completed the input that has completed
-   * @param transform the transform that has completed processing the input
-   * @param outputs the bundles the transform has output
-   * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
-   *                     is no hold
-   */
-  public void updateWatermarks(
-      @Nullable CommittedBundle<?> completed,
-      AppliedPTransform<?, ?, ?> transform,
-      TimerUpdate timerUpdate,
-      Iterable<? extends CommittedBundle<?>> outputs,
-      @Nullable Instant earliestHold) {
-    updatePending(completed, transform, timerUpdate, outputs);
-    TransformWatermarks transformWms = transformToWatermarks.get(transform);
-    transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
-    refreshWatermarks(transform);
-  }
-
-  private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    TransformWatermarks myWatermarks = transformToWatermarks.get(transform);
-    WatermarkUpdate updateResult = myWatermarks.refresh();
-    if (updateResult.isAdvanced()) {
-      for (PValue outputPValue : transform.getOutput().expand()) {
-        Collection<AppliedPTransform<?, ?, ?>> downstreamTransforms = consumers.get(outputPValue);
-        if (downstreamTransforms != null) {
-          for (AppliedPTransform<?, ?, ?> downstreamTransform : downstreamTransforms) {
-            refreshWatermarks(downstreamTransform);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Removes all of the completed Timers from the collection of pending timers, adds all new timers,
-   * and removes all deleted timers. Removes all elements consumed by the input bundle from the
-   * {@link PTransform PTransforms} collection of pending elements, and adds all elements produced
-   * by the {@link PTransform} to the pending queue of each consumer.
-   */
-  private void updatePending(
-      CommittedBundle<?> input,
-      AppliedPTransform<?, ?, ?> transform,
-      TimerUpdate timerUpdate,
-      Iterable<? extends CommittedBundle<?>> outputs) {
-    TransformWatermarks completedTransform = transformToWatermarks.get(transform);
-    completedTransform.updateTimers(timerUpdate);
-    if (input != null) {
-      completedTransform.removePending(input);
-    }
-
-    for (CommittedBundle<?> bundle : outputs) {
-      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
-        TransformWatermarks watermarks = transformToWatermarks.get(consumer);
-        watermarks.addPending(bundle);
-      }
-    }
-  }
-
-  /**
-   * Returns a map of each {@link PTransform} that has pending timers to those timers. All of the
-   * pending timers will be removed from this {@link InMemoryWatermarkManager}.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> allTimers = new HashMap<>();
-    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
-        transformToWatermarks.entrySet()) {
-      Map<Object, FiredTimers> keyFiredTimers = watermarksEntry.getValue().extractFiredTimers();
-      if (!keyFiredTimers.isEmpty()) {
-        allTimers.put(watermarksEntry.getKey(), keyFiredTimers);
-      }
-    }
-    return allTimers;
-  }
-
-  /**
-   * A (key, Instant) pair that holds the watermark. Holds are per-key, but the watermark is global,
-   * and as such the watermark manager must track holds and the release of holds on a per-key basis.
-   *
-   * <p>The {@link #compareTo(KeyedHold)} method of {@link KeyedHold} is not consistent with equals,
-   * as the key is arbitrarily ordered via identity, rather than object equality.
-   */
-  private static final class KeyedHold implements Comparable<KeyedHold> {
-    private static final Ordering<Object> KEY_ORDERING = Ordering.arbitrary().nullsLast();
-
-    private final Object key;
-    private final Instant timestamp;
-
-    /**
-     * Create a new KeyedHold with the specified key and timestamp.
-     */
-    public static KeyedHold of(Object key, Instant timestamp) {
-      return new KeyedHold(key, MoreObjects.firstNonNull(timestamp, THE_END_OF_TIME.get()));
-    }
-
-    private KeyedHold(Object key, Instant timestamp) {
-      this.key = key;
-      this.timestamp = timestamp;
-    }
-
-    @Override
-    public int compareTo(KeyedHold that) {
-      return ComparisonChain.start()
-          .compare(this.timestamp, that.timestamp)
-          .compare(this.key, that.key, KEY_ORDERING)
-          .result();
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(timestamp, key);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == null || !(other instanceof KeyedHold)) {
-        return false;
-      }
-      KeyedHold that = (KeyedHold) other;
-      return Objects.equals(this.timestamp, that.timestamp) && Objects.equals(this.key, that.key);
-    }
-
-    /**
-     * Get the value of this {@link KeyedHold}.
-     */
-    public Instant getTimestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(KeyedHold.class)
-          .add("key", key)
-          .add("hold", timestamp)
-          .toString();
-    }
-  }
-
-  private static class PerKeyHolds {
-    private final Map<Object, KeyedHold> keyedHolds;
-    private final PriorityQueue<KeyedHold> allHolds;
-
-    private PerKeyHolds() {
-      this.keyedHolds = new HashMap<>();
-      this.allHolds = new PriorityQueue<>();
-    }
-
-    /**
-     * Gets the minimum hold across all keys in this {@link PerKeyHolds}, or THE_END_OF_TIME if
-     * there are no holds within this {@link PerKeyHolds}.
-     */
-    public Instant getMinHold() {
-      return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
-    }
-
-    /**
-     * Updates the hold of the provided key to the provided value, removing any other holds for
-     * the same key.
-     */
-    public void updateHold(@Nullable Object key, Instant newHold) {
-      removeHold(key);
-      KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
-      keyedHolds.put(key, newKeyedHold);
-      allHolds.offer(newKeyedHold);
-    }
-
-    /**
-     * Removes the hold of the provided key.
-     */
-    public void removeHold(Object key) {
-      KeyedHold oldHold = keyedHolds.get(key);
-      if (oldHold != null) {
-        allHolds.remove(oldHold);
-      }
-    }
-  }
-
-  /**
-   * A reference to the input and output watermarks of an {@link AppliedPTransform}.
-   */
-  public class TransformWatermarks {
-    private final AppliedPTransformInputWatermark inputWatermark;
-    private final AppliedPTransformOutputWatermark outputWatermark;
-
-    private final SynchronizedProcessingTimeInputWatermark synchronizedProcessingInputWatermark;
-    private final SynchronizedProcessingTimeOutputWatermark synchronizedProcessingOutputWatermark;
-
-    private Instant latestSynchronizedInputWm;
-    private Instant latestSynchronizedOutputWm;
-
-    private TransformWatermarks(
-        AppliedPTransformInputWatermark inputWatermark,
-        AppliedPTransformOutputWatermark outputWatermark,
-        SynchronizedProcessingTimeInputWatermark inputSynchProcessingWatermark,
-        SynchronizedProcessingTimeOutputWatermark outputSynchProcessingWatermark) {
-      this.inputWatermark = inputWatermark;
-      this.outputWatermark = outputWatermark;
-
-      this.synchronizedProcessingInputWatermark = inputSynchProcessingWatermark;
-      this.synchronizedProcessingOutputWatermark = outputSynchProcessingWatermark;
-      this.latestSynchronizedInputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      this.latestSynchronizedOutputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Returns the input watermark of the {@link AppliedPTransform}.
-     */
-    public Instant getInputWatermark() {
-      return Preconditions.checkNotNull(inputWatermark.get());
-    }
-
-    /**
-     * Returns the output watermark of the {@link AppliedPTransform}.
-     */
-    public Instant getOutputWatermark() {
-      return outputWatermark.get();
-    }
-
-    /**
-     * Returns the synchronized processing input time of the {@link AppliedPTransform}.
-     *
-     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
-     * presence of holds, will increase as the system time progresses.
-     */
-    public synchronized Instant getSynchronizedProcessingInputTime() {
-      latestSynchronizedInputWm = INSTANT_ORDERING.max(
-          latestSynchronizedInputWm,
-          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingInputWatermark.get()));
-      return latestSynchronizedInputWm;
-    }
-
-    /**
-     * Returns the synchronized processing output time of the {@link AppliedPTransform}.
-     *
-     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
-     * presence of holds, will increase as the system time progresses.
-     */
-    public synchronized Instant getSynchronizedProcessingOutputTime() {
-      latestSynchronizedOutputWm = INSTANT_ORDERING.max(
-          latestSynchronizedOutputWm,
-          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingOutputWatermark.get()));
-      return latestSynchronizedOutputWm;
-    }
-
-    private WatermarkUpdate refresh() {
-      inputWatermark.refresh();
-      synchronizedProcessingInputWatermark.refresh();
-      WatermarkUpdate eventOutputUpdate = outputWatermark.refresh();
-      WatermarkUpdate syncOutputUpdate = synchronizedProcessingOutputWatermark.refresh();
-      return eventOutputUpdate.union(syncOutputUpdate);
-    }
-
-    private void setEventTimeHold(Object key, Instant newHold) {
-      outputWatermark.updateHold(key, newHold);
-    }
-
-    private void removePending(CommittedBundle<?> bundle) {
-      inputWatermark.removePendingElements(bundle.getElements());
-      synchronizedProcessingInputWatermark.removePending(bundle);
-    }
-
-    private void addPending(CommittedBundle<?> bundle) {
-      inputWatermark.addPendingElements(bundle.getElements());
-      synchronizedProcessingInputWatermark.addPending(bundle);
-    }
-
-    private Map<Object, FiredTimers> extractFiredTimers() {
-      Map<Object, List<TimerData>> eventTimeTimers = inputWatermark.extractFiredEventTimeTimers();
-      Map<Object, List<TimerData>> processingTimers;
-      Map<Object, List<TimerData>> synchronizedTimers;
-      if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
-        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
-        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
-      } else {
-        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, clock.now());
-        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime());
-      }
-      Map<Object, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
-      groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
-
-      Map<Object, FiredTimers> keyFiredTimers = new HashMap<>();
-      for (Map.Entry<Object, Map<TimeDomain, List<TimerData>>> firedTimers :
-          groupedTimers.entrySet()) {
-        keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
-      }
-      return keyFiredTimers;
-    }
-
-    @SafeVarargs
-    private final void groupFiredTimers(
-        Map<Object, Map<TimeDomain, List<TimerData>>> groupedToMutate,
-        Map<Object, List<TimerData>>... timersToGroup) {
-      for (Map<Object, List<TimerData>> subGroup : timersToGroup) {
-        for (Map.Entry<Object, List<TimerData>> newTimers : subGroup.entrySet()) {
-          Map<TimeDomain, List<TimerData>> grouped = groupedToMutate.get(newTimers.getKey());
-          if (grouped == null) {
-            grouped = new HashMap<>();
-            groupedToMutate.put(newTimers.getKey(), grouped);
-          }
-          grouped.put(newTimers.getValue().get(0).getDomain(), newTimers.getValue());
-        }
-      }
-    }
-
-    private void updateTimers(TimerUpdate update) {
-      inputWatermark.updateTimers(update);
-      synchronizedProcessingInputWatermark.updateTimers(update);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(TransformWatermarks.class)
-          .add("inputWatermark", inputWatermark)
-          .add("outputWatermark", outputWatermark)
-          .add("inputProcessingTime", synchronizedProcessingInputWatermark)
-          .add("outputProcessingTime", synchronizedProcessingOutputWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * A collection of newly set, deleted, and completed timers.
-   *
-   * <p>setTimers and deletedTimers are collections of {@link TimerData} that have been added to the
-   * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as
-   * the input to the executed step.
-   */
-  public static class TimerUpdate {
-    private final Object key;
-    private final Iterable<? extends TimerData> completedTimers;
-
-    private final Iterable<? extends TimerData> setTimers;
-    private final Iterable<? extends TimerData> deletedTimers;
-
-    /**
-     * Returns a TimerUpdate for a null key with no timers.
-     */
-    public static TimerUpdate empty() {
-      return new TimerUpdate(
-          null,
-          Collections.<TimerData>emptyList(),
-          Collections.<TimerData>emptyList(),
-          Collections.<TimerData>emptyList());
-    }
-
-    /**
-     * Creates a new {@link TimerUpdate} builder with the provided completed timers that needs the
-     * set and deleted timers to be added to it.
-     */
-    public static TimerUpdateBuilder builder(Object key) {
-      return new TimerUpdateBuilder(key);
-    }
-
-    /**
-     * A {@link TimerUpdate} builder that needs to be provided with set timers and deleted timers.
-     */
-    public static final class TimerUpdateBuilder {
-      private final Object key;
-      private final Collection<TimerData> completedTimers;
-      private final Collection<TimerData> setTimers;
-      private final Collection<TimerData> deletedTimers;
-
-      private TimerUpdateBuilder(Object key) {
-        this.key = key;
-        this.completedTimers = new HashSet<>();
-        this.setTimers = new HashSet<>();
-        this.deletedTimers = new HashSet<>();
-      }
-
-      /**
-       * Adds all of the provided timers to the collection of completed timers, and returns this
-       * {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder withCompletedTimers(Iterable<TimerData> completedTimers) {
-        Iterables.addAll(this.completedTimers, completedTimers);
-        return this;
-      }
-
-      /**
-       * Adds the provided timer to the collection of set timers, removing it from deleted timers if
-       * it has previously been deleted. Returns this {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder setTimer(TimerData setTimer) {
-        deletedTimers.remove(setTimer);
-        setTimers.add(setTimer);
-        return this;
-      }
-
-      /**
-       * Adds the provided timer to the collection of deleted timers, removing it from set timers if
-       * it has previously been set. Returns this {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
-        deletedTimers.add(deletedTimer);
-        setTimers.remove(deletedTimer);
-        return this;
-      }
-
-      /**
-       * Returns a new {@link TimerUpdate} with the most recently set completedTimers, setTimers,
-       * and deletedTimers.
-       */
-      public TimerUpdate build() {
-        return new TimerUpdate(
-            key,
-            ImmutableSet.copyOf(completedTimers),
-            ImmutableSet.copyOf(setTimers),
-            ImmutableSet.copyOf(deletedTimers));
-      }
-    }
-
-    private TimerUpdate(
-        Object key,
-        Iterable<? extends TimerData> completedTimers,
-        Iterable<? extends TimerData> setTimers,
-        Iterable<? extends TimerData> deletedTimers) {
-      this.key = key;
-      this.completedTimers = completedTimers;
-      this.setTimers = setTimers;
-      this.deletedTimers = deletedTimers;
-    }
-
-    @VisibleForTesting
-    Object getKey() {
-      return key;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getCompletedTimers() {
-      return completedTimers;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getSetTimers() {
-      return setTimers;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getDeletedTimers() {
-      return deletedTimers;
-    }
-
-    /**
-     * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
-     */
-    public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
-      return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(key, completedTimers, setTimers, deletedTimers);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == null || !(other instanceof TimerUpdate)) {
-        return false;
-      }
-      TimerUpdate that = (TimerUpdate) other;
-      return Objects.equals(this.key, that.key)
-          && Objects.equals(this.completedTimers, that.completedTimers)
-          && Objects.equals(this.setTimers, that.setTimers)
-          && Objects.equals(this.deletedTimers, that.deletedTimers);
-    }
-  }
-
-  /**
-   * A pair of {@link TimerData} and key which can be delivered to the appropriate
-   * {@link AppliedPTransform}. A timer fires at the transform that set it with a specific key when
-   * the time domain in which it lives progresses past a specified time, as determined by the
-   * {@link InMemoryWatermarkManager}.
-   */
-  public static class FiredTimers {
-    private final Map<TimeDomain, ? extends Collection<TimerData>> timers;
-
-    private FiredTimers(Map<TimeDomain, ? extends Collection<TimerData>> timers) {
-      this.timers = timers;
-    }
-
-    /**
-     * Gets all of the timers that have fired within the provided {@link TimeDomain}. If no timers
-     * fired within the provided domain, return an empty collection.
-     *
-     * <p>Timers within a {@link TimeDomain} are guaranteed to be in order of increasing timestamp.
-     */
-    public Collection<TimerData> getTimers(TimeDomain domain) {
-      Collection<TimerData> domainTimers = timers.get(domain);
-      if (domainTimers == null) {
-        return Collections.emptyList();
-      }
-      return domainTimers;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString();
-    }
-  }
-
-  private static class WindowedValueByTimestampComparator extends Ordering<WindowedValue<?>> {
-    @Override
-    public int compare(WindowedValue<?> o1, WindowedValue<?> o2) {
-      return o1.getTimestamp().compareTo(o2.getTimestamp());
-    }
-  }
-
-  public Set<AppliedPTransform<?, ?, ?>> getCompletedTransforms() {
-    Set<AppliedPTransform<?, ?, ?>> result = new HashSet<>();
-    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> wms :
-        transformToWatermarks.entrySet()) {
-      if (wms.getValue().getOutputWatermark().equals(THE_END_OF_TIME.get())) {
-        result.add(wms.getKey());
-      }
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundle.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundle.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundle.java
deleted file mode 100644
index 2fb6536..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundle.java
+++ /dev/null
@@ -1,123 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link UncommittedBundle} that buffers elements in memory.
- */
-public final class InProcessBundle<T> implements UncommittedBundle<T> {
-  private final PCollection<T> pcollection;
-  private final boolean keyed;
-  private final Object key;
-  private boolean committed = false;
-  private ImmutableList.Builder<WindowedValue<T>> elements;
-
-  /**
-   * Create a new {@link InProcessBundle} for the specified {@link PCollection} without a key.
-   */
-  public static <T> InProcessBundle<T> unkeyed(PCollection<T> pcollection) {
-    return new InProcessBundle<T>(pcollection, false, null);
-  }
-
-  /**
-   * Create a new {@link InProcessBundle} for the specified {@link PCollection} with the specified
-   * key.
-   *
-   * See {@link CommittedBundle#getKey()} and {@link CommittedBundle#isKeyed()} for more
-   * information.
-   */
-  public static <T> InProcessBundle<T> keyed(PCollection<T> pcollection, Object key) {
-    return new InProcessBundle<T>(pcollection, true, key);
-  }
-
-  private InProcessBundle(PCollection<T> pcollection, boolean keyed, Object key) {
-    this.pcollection = pcollection;
-    this.keyed = keyed;
-    this.key = key;
-    this.elements = ImmutableList.builder();
-  }
-
-  @Override
-  public PCollection<T> getPCollection() {
-    return pcollection;
-  }
-
-  @Override
-  public InProcessBundle<T> add(WindowedValue<T> element) {
-    checkState(!committed, "Can't add element %s to committed bundle %s", element, this);
-    elements.add(element);
-    return this;
-  }
-
-  @Override
-  public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
-    checkState(!committed, "Can't commit already committed bundle %s", this);
-    committed = true;
-    final Iterable<WindowedValue<T>> committedElements = elements.build();
-    return new CommittedBundle<T>() {
-      @Override
-      @Nullable
-      public Object getKey() {
-        return key;
-      }
-
-      @Override
-      public boolean isKeyed() {
-        return keyed;
-      }
-
-      @Override
-      public Iterable<WindowedValue<T>> getElements() {
-        return committedElements;
-      }
-
-      @Override
-      public PCollection<T> getPCollection() {
-        return pcollection;
-      }
-
-      @Override
-      public Instant getSynchronizedProcessingOutputWatermark() {
-        return synchronizedCompletionTime;
-      }
-
-      @Override
-      public String toString() {
-        return MoreObjects.toStringHelper(this)
-            .omitNullValues()
-            .add("pcollection", pcollection)
-            .add("key", key)
-            .add("elements", committedElements)
-            .toString();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleFactory.java
deleted file mode 100644
index 7ca1b60..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleFactory.java
+++ /dev/null
@@ -1,157 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory that produces bundles that perform no additional validation.
- */
-class InProcessBundleFactory implements BundleFactory {
-  public static InProcessBundleFactory create() {
-    return new InProcessBundleFactory();
-  }
-
-  private InProcessBundleFactory() {}
-
-  @Override
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return InProcessBundle.unkeyed(output);
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return input.isKeyed()
-        ? InProcessBundle.keyed(output, input.getKey())
-        : InProcessBundle.unkeyed(output);
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output) {
-    return InProcessBundle.keyed(output, key);
-  }
-
-  /**
-   * A {@link UncommittedBundle} that buffers elements in memory.
-   */
-  private static final class InProcessBundle<T> implements UncommittedBundle<T> {
-    private final PCollection<T> pcollection;
-    private final boolean keyed;
-    private final Object key;
-    private boolean committed = false;
-    private ImmutableList.Builder<WindowedValue<T>> elements;
-
-    /**
-     * Create a new {@link InProcessBundle} for the specified {@link PCollection} without a key.
-     */
-    public static <T> InProcessBundle<T> unkeyed(PCollection<T> pcollection) {
-      return new InProcessBundle<T>(pcollection, false, null);
-    }
-
-    /**
-     * Create a new {@link InProcessBundle} for the specified {@link PCollection} with the specified
-     * key.
-     *
-     * <p>See {@link CommittedBundle#getKey()} and {@link CommittedBundle#isKeyed()} for more
-     * information.
-     */
-    public static <T> InProcessBundle<T> keyed(PCollection<T> pcollection, Object key) {
-      return new InProcessBundle<T>(pcollection, true, key);
-    }
-
-    private InProcessBundle(PCollection<T> pcollection, boolean keyed, Object key) {
-      this.pcollection = pcollection;
-      this.keyed = keyed;
-      this.key = key;
-      this.elements = ImmutableList.builder();
-    }
-
-    @Override
-    public PCollection<T> getPCollection() {
-      return pcollection;
-    }
-
-    @Override
-    public InProcessBundle<T> add(WindowedValue<T> element) {
-      checkState(
-          !committed,
-          "Can't add element %s to committed bundle in PCollection %s",
-          element,
-          pcollection);
-      elements.add(element);
-      return this;
-    }
-
-    @Override
-    public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
-      checkState(!committed, "Can't commit already committed bundle %s", this);
-      committed = true;
-      final Iterable<WindowedValue<T>> committedElements = elements.build();
-      return new CommittedBundle<T>() {
-        @Override
-        @Nullable
-        public Object getKey() {
-          return key;
-        }
-
-        @Override
-        public boolean isKeyed() {
-          return keyed;
-        }
-
-        @Override
-        public Iterable<WindowedValue<T>> getElements() {
-          return committedElements;
-        }
-
-        @Override
-        public PCollection<T> getPCollection() {
-          return pcollection;
-        }
-
-        @Override
-        public Instant getSynchronizedProcessingOutputWatermark() {
-          return synchronizedCompletionTime;
-        }
-
-        @Override
-        public String toString() {
-          return MoreObjects.toStringHelper(this)
-              .omitNullValues()
-              .add("pcollection", pcollection)
-              .add("key", key)
-              .add("elements", committedElements)
-              .toString();
-        }
-      };
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleOutputManager.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleOutputManager.java
deleted file mode 100644
index de6a3dc..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessBundleOutputManager.java
+++ /dev/null
@@ -1,52 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-import java.util.Map;
-
-/**
- * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessBundleOutputManager implements OutputManager {
-  private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
-
-  public static InProcessBundleOutputManager create(
-      Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
-    return new InProcessBundleOutputManager(outputBundles);
-  }
-
-  public InProcessBundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
-    this.bundles = bundles;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-    @SuppressWarnings("rawtypes")
-    UncommittedBundle bundle = bundles.get(tag);
-    bundle.add(output);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessCreate.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessCreate.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessCreate.java
deleted file mode 100644
index 28b4de7..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessCreate.java
+++ /dev/null
@@ -1,235 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.OffsetBasedSource;
-import com.google.cloud.dataflow.sdk.io.OffsetBasedSource.OffsetBasedReader;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Create.Values;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.util.CoderUtils;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-
-/**
- * An in-process implementation of the {@link Values Create.Values} {@link PTransform}, implemented
- * using a {@link BoundedSource}.
- *
- * The coder is inferred via the {@link Values#getDefaultOutputCoder(PInput)} method on the original
- * transform.
- */
-class InProcessCreate<T> extends ForwardingPTransform<PInput, PCollection<T>> {
-  private final Create.Values<T> original;
-
-  /**
-   * A {@link PTransformOverrideFactory} for {@link InProcessCreate}.
-   */
-  public static class InProcessCreateOverrideFactory implements PTransformOverrideFactory {
-    @Override
-    public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-        PTransform<InputT, OutputT> transform) {
-      if (transform instanceof Create.Values) {
-        @SuppressWarnings("unchecked")
-        PTransform<InputT, OutputT> override =
-            (PTransform<InputT, OutputT>) from((Create.Values<OutputT>) transform);
-        return override;
-      }
-      return transform;
-    }
-  }
-
-  public static <T> InProcessCreate<T> from(Create.Values<T> original) {
-    return new InProcessCreate<>(original);
-  }
-
-  private InProcessCreate(Values<T> original) {
-    this.original = original;
-  }
-
-  @Override
-  public PCollection<T> apply(PInput input) {
-    Coder<T> elementCoder;
-    try {
-      elementCoder = original.getDefaultOutputCoder(input);
-    } catch (CannotProvideCoderException e) {
-      throw new IllegalArgumentException(
-          "Unable to infer a coder and no Coder was specified. "
-          + "Please set a coder by invoking Create.withCoder() explicitly.",
-          e);
-    }
-    InMemorySource<T> source;
-    try {
-      source = InMemorySource.fromIterable(original.getElements(), elementCoder);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    PCollection<T> result = input.getPipeline().apply(Read.from(source));
-    result.setCoder(elementCoder);
-    return result;
-  }
-
-  @Override
-  public PTransform<PInput, PCollection<T>> delegate() {
-    return original;
-  }
-
-  @VisibleForTesting
-  static class InMemorySource<T> extends OffsetBasedSource<T> {
-    private final List<byte[]> allElementsBytes;
-    private final long totalSize;
-    private final Coder<T> coder;
-
-    public static <T> InMemorySource<T> fromIterable(Iterable<T> elements, Coder<T> elemCoder)
-        throws CoderException, IOException {
-      ImmutableList.Builder<byte[]> allElementsBytes = ImmutableList.builder();
-      long totalSize = 0L;
-      for (T element : elements) {
-        byte[] bytes = CoderUtils.encodeToByteArray(elemCoder, element);
-        allElementsBytes.add(bytes);
-        totalSize += bytes.length;
-      }
-      return new InMemorySource<>(allElementsBytes.build(), totalSize, elemCoder);
-    }
-
-    /**
-     * Create a new source with the specified bytes. The new source owns the input element bytes,
-     * which must not be modified after this constructor is called.
-     */
-    private InMemorySource(List<byte[]> elementBytes, long totalSize, Coder<T> coder) {
-      super(0, elementBytes.size(), 1);
-      this.allElementsBytes = ImmutableList.copyOf(elementBytes);
-      this.totalSize = totalSize;
-      this.coder = coder;
-    }
-
-    @Override
-    public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
-      return totalSize;
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-      return false;
-    }
-
-    @Override
-    public BoundedSource.BoundedReader<T> createReader(PipelineOptions options) throws IOException {
-      return new BytesReader<>(this);
-    }
-
-    @Override
-    public void validate() {}
-
-    @Override
-    public Coder<T> getDefaultOutputCoder() {
-      return coder;
-    }
-
-    @Override
-    public long getMaxEndOffset(PipelineOptions options) throws Exception {
-      return allElementsBytes.size();
-    }
-
-    @Override
-    public OffsetBasedSource<T> createSourceForSubrange(long start, long end) {
-      List<byte[]> primaryElems = allElementsBytes.subList((int) start, (int) end);
-      long primarySizeEstimate =
-          (long) (totalSize * primaryElems.size() / (double) allElementsBytes.size());
-      return new InMemorySource<>(primaryElems, primarySizeEstimate, coder);
-    }
-
-    @Override
-    public long getBytesPerOffset() {
-      if (allElementsBytes.size() == 0) {
-        return 0L;
-      }
-      return totalSize / allElementsBytes.size();
-    }
-  }
-
-  private static class BytesReader<T> extends OffsetBasedReader<T> {
-    private int index;
-    /**
-     * Use an optional to distinguish between null next element (as Optional.absent()) and no next
-     * element (next is null).
-     */
-    @Nullable private Optional<T> next;
-
-    public BytesReader(InMemorySource<T> source) {
-      super(source);
-      index = -1;
-    }
-
-    @Override
-    @Nullable
-    public T getCurrent() throws NoSuchElementException {
-      if (next == null) {
-        throw new NoSuchElementException();
-      }
-      return next.orNull();
-    }
-
-    @Override
-    public void close() throws IOException {}
-
-    @Override
-    protected long getCurrentOffset() {
-      return index;
-    }
-
-    @Override
-    protected boolean startImpl() throws IOException {
-      return advanceImpl();
-    }
-
-    @Override
-    public synchronized InMemorySource<T> getCurrentSource() {
-      return (InMemorySource<T>) super.getCurrentSource();
-    }
-
-    @Override
-    protected boolean advanceImpl() throws IOException {
-      InMemorySource<T> source = getCurrentSource();
-      index++;
-      if (index >= source.allElementsBytes.size()) {
-        return false;
-      }
-      next =
-          Optional.fromNullable(
-              CoderUtils.decodeFromByteArray(
-                  source.coder, source.allElementsBytes.get(index)));
-      return true;
-    }
-  }
-}


[73/74] incubator-beam git commit: Fix test ordering in DataflowPipelineTranslatorTest

Posted by da...@apache.org.
Fix test ordering in DataflowPipelineTranslatorTest


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

Branch: refs/heads/master
Commit: 75a1905ebe09755da5c464a72aafc67bc982b0bc
Parents: b972445
Author: bchambers <bc...@google.com>
Authored: Wed Apr 13 17:39:34 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Wed Apr 13 21:24:28 2016 -0700

----------------------------------------------------------------------
 .../runners/DataflowPipelineTranslatorTest.java | 71 +++++++++++---------
 1 file changed, 38 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a1905e/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
index 97d1a50..1429e5a 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
@@ -74,6 +74,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
+import org.hamcrest.Matcher;
 import org.hamcrest.Matchers;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -854,39 +855,43 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Map<String, Object> parDo2Properties = steps.get(2).getProperties();
     assertThat(parDo1Properties, hasKey("display_data"));
 
-    Collection<Map<String, String>> fn1displayData =
-            (Collection<Map<String, String>>) parDo1Properties.get("display_data");
-    Collection<Map<String, String>> fn2displayData =
-            (Collection<Map<String, String>>) parDo2Properties.get("display_data");
-
-    ImmutableList expectedFn1DisplayData = ImmutableList.of(
-            ImmutableMap.<String, String>builder()
-              .put("namespace", fn1.getClass().getName())
-              .put("key", "foo")
-              .put("type", "STRING")
-              .put("value", "bar")
-              .build(),
-            ImmutableMap.<String, String>builder()
-              .put("namespace", fn1.getClass().getName())
-              .put("key", "foo2")
-              .put("type", "JAVA_CLASS")
-              .put("value", DataflowPipelineTranslatorTest.class.getName())
-              .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
-              .put("label", "Test Class")
-              .put("linkUrl", "http://www.google.com")
-              .build()
-    );
-
-    ImmutableList expectedFn2DisplayData = ImmutableList.of(
+
+    @SuppressWarnings("unchecked")
+    Collection<Map<String, Object>> fn1displayData =
+            (Collection<Map<String, Object>>) parDo1Properties.get("display_data");
+    @SuppressWarnings("unchecked")
+    Collection<Map<String, Object>> fn2displayData =
+            (Collection<Map<String, Object>>) parDo2Properties.get("display_data");
+
+    @SuppressWarnings("unchecked")
+    Matcher<Iterable<? extends Map<String, Object>>> fn1expectedData =
+        Matchers.<Map<String, Object>>containsInAnyOrder(
+            ImmutableMap.<String, Object>builder()
+                .put("namespace", fn1.getClass().getName())
+                .put("key", "foo")
+                .put("type", "STRING")
+                .put("value", "bar")
+                .build(),
+            ImmutableMap.<String, Object>builder()
+                .put("namespace", fn1.getClass().getName())
+                .put("key", "foo2")
+                .put("type", "JAVA_CLASS")
+                .put("value", DataflowPipelineTranslatorTest.class.getName())
+                .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
+                .put("label", "Test Class")
+                .put("linkUrl", "http://www.google.com")
+                .build());
+
+    @SuppressWarnings("unchecked")
+    Matcher<Iterable<? extends Map<String, Object>>> fn2expectedData =
+        Matchers.<Map<String, Object>>contains(
             ImmutableMap.<String, Object>builder()
-                    .put("namespace", fn2.getClass().getName())
-                    .put("key", "foo3")
-                    .put("type", "INTEGER")
-                    .put("value", 1234L)
-                    .build()
-    );
-
-    assertEquals(expectedFn1DisplayData, fn1displayData);
-    assertEquals(expectedFn2DisplayData, fn2displayData);
+                .put("namespace", fn2.getClass().getName())
+                .put("key", "foo3")
+                .put("type", "INTEGER")
+                .put("value", 1234L)
+                .build());
+    assertThat(fn1displayData, fn1expectedData);
+    assertThat(fn2displayData, fn2expectedData);
   }
 }


[15/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java
deleted file mode 100644
index 3ad32b4..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java
+++ /dev/null
@@ -1,663 +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 com.google.cloud.dataflow.sdk.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.util.IOChannelFactory;
-import com.google.cloud.dataflow.sdk.util.IOChannelUtils;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.NoSuchElementException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-
-/**
- * A common base class for all file-based {@link Source}s. Extend this class to implement your own
- * file-based custom source.
- *
- * <p>A file-based {@code Source} is a {@code Source} backed by a file pattern defined as a Java
- * glob, a single file, or a offset range for a single file. See {@link OffsetBasedSource} and
- * {@link com.google.cloud.dataflow.sdk.io.range.RangeTracker} for semantics of offset ranges.
- *
- * <p>This source stores a {@code String} that is an {@link IOChannelFactory} specification for a
- * file or file pattern. There should be an {@code IOChannelFactory} defined for the file
- * specification provided. Please refer to {@link IOChannelUtils} and {@link IOChannelFactory} for
- * more information on this.
- *
- * <p>In addition to the methods left abstract from {@code BoundedSource}, subclasses must implement
- * methods to create a sub-source and a reader for a range of a single file -
- * {@link #createForSubrangeOfFile} and {@link #createSingleFileReader}. Please refer to
- * {@link XmlSource} for an example implementation of {@code FileBasedSource}.
- *
- * @param <T> Type of records represented by the source.
- */
-public abstract class FileBasedSource<T> extends OffsetBasedSource<T> {
-  private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class);
-  private static final float FRACTION_OF_FILES_TO_STAT = 0.01f;
-
-  // Package-private for testing
-  static final int MAX_NUMBER_OF_FILES_FOR_AN_EXACT_STAT = 100;
-
-  // Size of the thread pool to be used for performing file operations in parallel.
-  // Package-private for testing.
-  static final int THREAD_POOL_SIZE = 128;
-
-  private final String fileOrPatternSpec;
-  private final Mode mode;
-
-  /**
-   * A given {@code FileBasedSource} represents a file resource of one of these types.
-   */
-  public enum Mode {
-    FILEPATTERN,
-    SINGLE_FILE_OR_SUBRANGE
-  }
-
-  /**
-   * Create a {@code FileBaseSource} based on a file or a file pattern specification. This
-   * constructor must be used when creating a new {@code FileBasedSource} for a file pattern.
-   *
-   * <p>See {@link OffsetBasedSource} for a detailed description of {@code minBundleSize}.
-   *
-   * @param fileOrPatternSpec {@link IOChannelFactory} specification of file or file pattern
-   *        represented by the {@link FileBasedSource}.
-   * @param minBundleSize minimum bundle size in bytes.
-   */
-  public FileBasedSource(String fileOrPatternSpec, long minBundleSize) {
-    super(0, Long.MAX_VALUE, minBundleSize);
-    mode = Mode.FILEPATTERN;
-    this.fileOrPatternSpec = fileOrPatternSpec;
-  }
-
-  /**
-   * Create a {@code FileBasedSource} based on a single file. This constructor must be used when
-   * creating a new {@code FileBasedSource} for a subrange of a single file.
-   * Additionally, this constructor must be used to create new {@code FileBasedSource}s when
-   * subclasses implement the method {@link #createForSubrangeOfFile}.
-   *
-   * <p>See {@link OffsetBasedSource} for detailed descriptions of {@code minBundleSize},
-   * {@code startOffset}, and {@code endOffset}.
-   *
-   * @param fileName {@link IOChannelFactory} specification of the file represented by the
-   *        {@link FileBasedSource}.
-   * @param minBundleSize minimum bundle size in bytes.
-   * @param startOffset starting byte offset.
-   * @param endOffset ending byte offset. If the specified value {@code >= #getMaxEndOffset()} it
-   *        implies {@code #getMaxEndOffSet()}.
-   */
-  public FileBasedSource(String fileName, long minBundleSize,
-      long startOffset, long endOffset) {
-    super(startOffset, endOffset, minBundleSize);
-    mode = Mode.SINGLE_FILE_OR_SUBRANGE;
-    this.fileOrPatternSpec = fileName;
-  }
-
-  public final String getFileOrPatternSpec() {
-    return fileOrPatternSpec;
-  }
-
-  public final Mode getMode() {
-    return mode;
-  }
-
-  @Override
-  public final FileBasedSource<T> createSourceForSubrange(long start, long end) {
-    checkArgument(mode != Mode.FILEPATTERN,
-        "Cannot split a file pattern based source based on positions");
-    checkArgument(start >= getStartOffset(),
-        "Start offset value %s of the subrange cannot be smaller than the start offset value %s"
-            + " of the parent source",
-        start,
-        getStartOffset());
-    checkArgument(end <= getEndOffset(),
-        "End offset value %s of the subrange cannot be larger than the end offset value %s",
-        end,
-        getEndOffset());
-
-    FileBasedSource<T> source = createForSubrangeOfFile(fileOrPatternSpec, start, end);
-    if (start > 0 || end != Long.MAX_VALUE) {
-      checkArgument(source.getMode() == Mode.SINGLE_FILE_OR_SUBRANGE,
-          "Source created for the range [%s,%s) must be a subrange source", start, end);
-    }
-    return source;
-  }
-
-  /**
-   * Creates and returns a new {@code FileBasedSource} of the same type as the current
-   * {@code FileBasedSource} backed by a given file and an offset range. When current source is
-   * being split, this method is used to generate new sub-sources. When creating the source
-   * subclasses must call the constructor {@link #FileBasedSource(String, long, long, long)} of
-   * {@code FileBasedSource} with corresponding parameter values passed here.
-   *
-   * @param fileName file backing the new {@code FileBasedSource}.
-   * @param start starting byte offset of the new {@code FileBasedSource}.
-   * @param end ending byte offset of the new {@code FileBasedSource}. May be Long.MAX_VALUE,
-   *        in which case it will be inferred using {@link #getMaxEndOffset}.
-   */
-  protected abstract FileBasedSource<T> createForSubrangeOfFile(
-      String fileName, long start, long end);
-
-  /**
-   * Creates and returns an instance of a {@code FileBasedReader} implementation for the current
-   * source assuming the source represents a single file. File patterns will be handled by
-   * {@code FileBasedSource} implementation automatically.
-   */
-  protected abstract FileBasedReader<T> createSingleFileReader(
-      PipelineOptions options);
-
-  @Override
-  public final long getEstimatedSizeBytes(PipelineOptions options) throws IOException {
-    // This implementation of method getEstimatedSizeBytes is provided to simplify subclasses. Here
-    // we perform the size estimation of files and file patterns using the interface provided by
-    // IOChannelFactory.
-
-    IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec);
-    if (mode == Mode.FILEPATTERN) {
-      // TODO Implement a more efficient parallel/batch size estimation mechanism for file patterns.
-      long startTime = System.currentTimeMillis();
-      long totalSize = 0;
-      Collection<String> inputs = factory.match(fileOrPatternSpec);
-      if (inputs.size() <= MAX_NUMBER_OF_FILES_FOR_AN_EXACT_STAT) {
-        totalSize = getExactTotalSizeOfFiles(inputs, factory);
-        LOG.debug("Size estimation of all files of pattern {} took {} ms",
-            fileOrPatternSpec,
-            System.currentTimeMillis() - startTime);
-      } else {
-        totalSize = getEstimatedSizeOfFilesBySampling(inputs, factory);
-        LOG.debug("Size estimation of pattern {} by sampling took {} ms",
-            fileOrPatternSpec,
-            System.currentTimeMillis() - startTime);
-      }
-      return totalSize;
-    } else {
-      long start = getStartOffset();
-      long end = Math.min(getEndOffset(), getMaxEndOffset(options));
-      return end - start;
-    }
-  }
-
-  // Get the exact total size of the given set of files.
-  // Invokes multiple requests for size estimation in parallel using a thread pool.
-  // TODO: replace this with bulk request API when it is available. Will require updates
-  // to IOChannelFactory interface.
-  private static long getExactTotalSizeOfFiles(
-      Collection<String> files, IOChannelFactory ioChannelFactory) throws IOException {
-    List<ListenableFuture<Long>> futures = new ArrayList<>();
-    ListeningExecutorService service =
-        MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(THREAD_POOL_SIZE));
-    try {
-      long totalSize = 0;
-      for (String file : files) {
-        futures.add(createFutureForSizeEstimation(file, ioChannelFactory, service));
-      }
-
-      for (Long val : Futures.allAsList(futures).get()) {
-        totalSize += val;
-      }
-
-      return totalSize;
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new IOException(e);
-    } catch (ExecutionException e) {
-      throw new IOException(e.getCause());
-    }  finally {
-      service.shutdown();
-    }
-  }
-
-  private static ListenableFuture<Long> createFutureForSizeEstimation(
-      final String file,
-      final IOChannelFactory ioChannelFactory,
-      ListeningExecutorService service) {
-    return service.submit(
-        new Callable<Long>() {
-          @Override
-          public Long call() throws IOException {
-            return ioChannelFactory.getSizeBytes(file);
-          }
-        });
-  }
-
-  // Estimate the total size of the given set of files through sampling and extrapolation.
-  // Currently we use uniform sampling which requires a linear sampling size for a reasonable
-  // estimate.
-  // TODO: Implement a more efficient sampling mechanism.
-  private static long getEstimatedSizeOfFilesBySampling(
-      Collection<String> files, IOChannelFactory ioChannelFactory) throws IOException {
-    int sampleSize = (int) (FRACTION_OF_FILES_TO_STAT * files.size());
-    sampleSize = Math.max(MAX_NUMBER_OF_FILES_FOR_AN_EXACT_STAT, sampleSize);
-
-    List<String> selectedFiles = new ArrayList<String>(files);
-    Collections.shuffle(selectedFiles);
-    selectedFiles = selectedFiles.subList(0, sampleSize);
-
-    return files.size() * getExactTotalSizeOfFiles(selectedFiles, ioChannelFactory)
-        / selectedFiles.size();
-  }
-
-  private ListenableFuture<List<? extends FileBasedSource<T>>> createFutureForFileSplit(
-      final String file,
-      final long desiredBundleSizeBytes,
-      final PipelineOptions options,
-      ListeningExecutorService service) {
-    return service.submit(new Callable<List<? extends FileBasedSource<T>>>() {
-      @Override
-      public List<? extends FileBasedSource<T>> call() throws Exception {
-        return createForSubrangeOfFile(file, 0, Long.MAX_VALUE)
-            .splitIntoBundles(desiredBundleSizeBytes, options);
-      }
-    });
-  }
-
-  @Override
-  public final List<? extends FileBasedSource<T>> splitIntoBundles(
-      long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
-    // This implementation of method splitIntoBundles is provided to simplify subclasses. Here we
-    // split a FileBasedSource based on a file pattern to FileBasedSources based on full single
-    // files. For files that can be efficiently seeked, we further split FileBasedSources based on
-    // those files to FileBasedSources based on sub ranges of single files.
-
-    if (mode == Mode.FILEPATTERN) {
-      long startTime = System.currentTimeMillis();
-      List<ListenableFuture<List<? extends FileBasedSource<T>>>> futures = new ArrayList<>();
-
-      ListeningExecutorService service =
-          MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(THREAD_POOL_SIZE));
-      try {
-        for (final String file : FileBasedSource.expandFilePattern(fileOrPatternSpec)) {
-          futures.add(createFutureForFileSplit(file, desiredBundleSizeBytes, options, service));
-        }
-        List<? extends FileBasedSource<T>> splitResults =
-            ImmutableList.copyOf(Iterables.concat(Futures.allAsList(futures).get()));
-        LOG.debug(
-            "Splitting the source based on file pattern {} took {} ms",
-            fileOrPatternSpec,
-            System.currentTimeMillis() - startTime);
-        return splitResults;
-      } finally {
-        service.shutdown();
-      }
-    } else {
-      if (isSplittable()) {
-        List<FileBasedSource<T>> splitResults = new ArrayList<>();
-        for (OffsetBasedSource<T> split :
-            super.splitIntoBundles(desiredBundleSizeBytes, options)) {
-          splitResults.add((FileBasedSource<T>) split);
-        }
-        return splitResults;
-      } else {
-        LOG.debug("The source for file {} is not split into sub-range based sources since "
-            + "the file is not seekable",
-            fileOrPatternSpec);
-        return ImmutableList.of(this);
-      }
-    }
-  }
-
-  /**
-   * Determines whether a file represented by this source is can be split into bundles.
-   *
-   * <p>By default, a file is splittable if it is on a file system that supports efficient read
-   * seeking. Subclasses may override to provide different behavior.
-   */
-  protected boolean isSplittable() throws Exception {
-    // We split a file-based source into subranges only if the file is efficiently seekable.
-    // If a file is not efficiently seekable it would be highly inefficient to create and read a
-    // source based on a subrange of that file.
-    IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec);
-    return factory.isReadSeekEfficient(fileOrPatternSpec);
-  }
-
-  @Override
-  public final BoundedReader<T> createReader(PipelineOptions options) throws IOException {
-    // Validate the current source prior to creating a reader for it.
-    this.validate();
-
-    if (mode == Mode.FILEPATTERN) {
-      long startTime = System.currentTimeMillis();
-      Collection<String> files = FileBasedSource.expandFilePattern(fileOrPatternSpec);
-      List<FileBasedReader<T>> fileReaders = new ArrayList<>();
-      for (String fileName : files) {
-        long endOffset;
-        try {
-          endOffset = IOChannelUtils.getFactory(fileName).getSizeBytes(fileName);
-        } catch (IOException e) {
-          LOG.warn("Failed to get size of {}", fileName, e);
-          endOffset = Long.MAX_VALUE;
-        }
-        fileReaders.add(
-            createForSubrangeOfFile(fileName, 0, endOffset).createSingleFileReader(options));
-      }
-      LOG.debug(
-          "Creating a reader for file pattern {} took {} ms",
-          fileOrPatternSpec,
-          System.currentTimeMillis() - startTime);
-      if (fileReaders.size() == 1) {
-        return fileReaders.get(0);
-      }
-      return new FilePatternReader(this, fileReaders);
-    } else {
-      return createSingleFileReader(options);
-    }
-  }
-
-  @Override
-  public String toString() {
-    switch (mode) {
-      case FILEPATTERN:
-        return fileOrPatternSpec;
-      case SINGLE_FILE_OR_SUBRANGE:
-        return fileOrPatternSpec + " range " + super.toString();
-      default:
-        throw new IllegalStateException("Unexpected mode: " + mode);
-    }
-  }
-
-  @Override
-  public void validate() {
-    super.validate();
-    switch (mode) {
-      case FILEPATTERN:
-        checkArgument(getStartOffset() == 0,
-            "FileBasedSource is based on a file pattern or a full single file "
-            + "but the starting offset proposed %s is not zero", getStartOffset());
-        checkArgument(getEndOffset() == Long.MAX_VALUE,
-            "FileBasedSource is based on a file pattern or a full single file "
-            + "but the ending offset proposed %s is not Long.MAX_VALUE", getEndOffset());
-        break;
-      case SINGLE_FILE_OR_SUBRANGE:
-        // Nothing more to validate.
-        break;
-      default:
-        throw new IllegalStateException("Unknown mode: " + mode);
-    }
-  }
-
-  @Override
-  public final long getMaxEndOffset(PipelineOptions options) throws IOException {
-    checkArgument(
-            mode != Mode.FILEPATTERN, "Cannot determine the exact end offset of a file pattern");
-    if (getEndOffset() == Long.MAX_VALUE) {
-      IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec);
-      return factory.getSizeBytes(fileOrPatternSpec);
-    } else {
-      return getEndOffset();
-    }
-  }
-
-  protected static final Collection<String> expandFilePattern(String fileOrPatternSpec)
-      throws IOException {
-    IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec);
-    Collection<String> matches = factory.match(fileOrPatternSpec);
-    LOG.info("Matched {} files for pattern {}", matches.size(), fileOrPatternSpec);
-    return matches;
-  }
-
-  /**
-   * A {@link Source.Reader reader} that implements code common to readers of
-   * {@code FileBasedSource}s.
-   *
-   * <h2>Seekability</h2>
-   *
-   * <p>This reader uses a {@link ReadableByteChannel} created for the file represented by the
-   * corresponding source to efficiently move to the correct starting position defined in the
-   * source. Subclasses of this reader should implement {@link #startReading} to get access to this
-   * channel. If the source corresponding to the reader is for a subrange of a file the
-   * {@code ReadableByteChannel} provided is guaranteed to be an instance of the type
-   * {@link SeekableByteChannel}, which may be used by subclass to traverse back in the channel to
-   * determine the correct starting position.
-   *
-   * <h2>Reading Records</h2>
-   *
-   * <p>Sequential reading is implemented using {@link #readNextRecord}.
-   *
-   * <p>Then {@code FileBasedReader} implements "reading a range [A, B)" in the following way.
-   * <ol>
-   * <li>{@link #start} opens the file
-   * <li>{@link #start} seeks the {@code SeekableByteChannel} to A (reading offset ranges for
-   * non-seekable files is not supported) and calls {@code startReading()}
-   * <li>{@link #start} calls {@link #advance} once, which, via {@link #readNextRecord},
-   * locates the first record which is at a split point AND its offset is at or after A.
-   * If this record is at or after B, {@link #advance} returns false and reading is finished.
-   * <li>if the previous advance call returned {@code true} sequential reading starts and
-   * {@code advance()} will be called repeatedly
-   * </ol>
-   * {@code advance()} calls {@code readNextRecord()} on the subclass, and stops (returns false) if
-   * the new record is at a split point AND the offset of the new record is at or after B.
-   *
-   * <h2>Thread Safety</h2>
-   *
-   * <p>Since this class implements {@link Source.Reader} it guarantees thread safety. Abstract
-   * methods defined here will not be accessed by more than one thread concurrently.
-   */
-  public abstract static class FileBasedReader<T> extends OffsetBasedReader<T> {
-    private ReadableByteChannel channel = null;
-
-    /**
-     * Subclasses should not perform IO operations at the constructor. All IO operations should be
-     * delayed until the {@link #startReading} method is invoked.
-     */
-    public FileBasedReader(FileBasedSource<T> source) {
-      super(source);
-      checkArgument(source.getMode() != Mode.FILEPATTERN,
-          "FileBasedReader does not support reading file patterns");
-    }
-
-    @Override
-    public FileBasedSource<T> getCurrentSource() {
-      return (FileBasedSource<T>) super.getCurrentSource();
-    }
-
-    @Override
-    protected final boolean startImpl() throws IOException {
-      FileBasedSource<T> source = getCurrentSource();
-      IOChannelFactory factory = IOChannelUtils.getFactory(source.getFileOrPatternSpec());
-      this.channel = factory.open(source.getFileOrPatternSpec());
-
-      if (channel instanceof SeekableByteChannel) {
-        SeekableByteChannel seekChannel = (SeekableByteChannel) channel;
-        seekChannel.position(source.getStartOffset());
-      } else {
-        // Channel is not seekable. Must not be a subrange.
-        checkArgument(source.mode != Mode.SINGLE_FILE_OR_SUBRANGE,
-            "Subrange-based sources must only be defined for file types that support seekable "
-            + " read channels");
-        checkArgument(source.getStartOffset() == 0,
-            "Start offset %s is not zero but channel for reading the file is not seekable.",
-            source.getStartOffset());
-      }
-
-      startReading(channel);
-
-      // Advance once to load the first record.
-      return advanceImpl();
-    }
-
-    @Override
-    protected final boolean advanceImpl() throws IOException {
-      return readNextRecord();
-    }
-
-    /**
-     * Closes any {@link ReadableByteChannel} created for the current reader. This implementation is
-     * idempotent. Any {@code close()} method introduced by a subclass must be idempotent and must
-     * call the {@code close()} method in the {@code FileBasedReader}.
-     */
-    @Override
-    public void close() throws IOException {
-      if (channel != null) {
-        channel.close();
-      }
-    }
-
-    /**
-     * Performs any initialization of the subclass of {@code FileBasedReader} that involves IO
-     * operations. Will only be invoked once and before that invocation the base class will seek the
-     * channel to the source's starting offset.
-     *
-     * <p>Provided {@link ReadableByteChannel} is for the file represented by the source of this
-     * reader. Subclass may use the {@code channel} to build a higher level IO abstraction, e.g., a
-     * BufferedReader or an XML parser.
-     *
-     * <p>If the corresponding source is for a subrange of a file, {@code channel} is guaranteed to
-     * be an instance of the type {@link SeekableByteChannel}.
-     *
-     * <p>After this method is invoked the base class will not be reading data from the channel or
-     * adjusting the position of the channel. But the base class is responsible for properly closing
-     * the channel.
-     *
-     * @param channel a byte channel representing the file backing the reader.
-     */
-    protected abstract void startReading(ReadableByteChannel channel) throws IOException;
-
-    /**
-     * Reads the next record from the channel provided by {@link #startReading}. Methods
-     * {@link #getCurrent}, {@link #getCurrentOffset}, and {@link #isAtSplitPoint()} should return
-     * the corresponding information about the record read by the last invocation of this method.
-     *
-     * <p>Note that this method will be called the same way for reading the first record in the
-     * source (file or offset range in the file) and for reading subsequent records. It is up to the
-     * subclass to do anything special for locating and reading the first record, if necessary.
-     *
-     * @return {@code true} if a record was successfully read, {@code false} if the end of the
-     *         channel was reached before successfully reading a new record.
-     */
-    protected abstract boolean readNextRecord() throws IOException;
-  }
-
-  // An internal Reader implementation that concatenates a sequence of FileBasedReaders.
-  private class FilePatternReader extends BoundedReader<T> {
-    private final FileBasedSource<T> source;
-    private final List<FileBasedReader<T>> fileReaders;
-    final ListIterator<FileBasedReader<T>> fileReadersIterator;
-    FileBasedReader<T> currentReader = null;
-
-    public FilePatternReader(FileBasedSource<T> source, List<FileBasedReader<T>> fileReaders) {
-      this.source = source;
-      this.fileReaders = fileReaders;
-      this.fileReadersIterator = fileReaders.listIterator();
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      return startNextNonemptyReader();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      checkState(currentReader != null, "Call start() before advance()");
-      if (currentReader.advance()) {
-        return true;
-      }
-      return startNextNonemptyReader();
-    }
-
-    private boolean startNextNonemptyReader() throws IOException {
-      while (fileReadersIterator.hasNext()) {
-        currentReader = fileReadersIterator.next();
-        if (currentReader.start()) {
-          return true;
-        }
-        currentReader.close();
-      }
-      return false;
-    }
-
-    @Override
-    public T getCurrent() throws NoSuchElementException {
-      // A NoSuchElement will be thrown by the last FileBasedReader if getCurrent() is called after
-      // advance() returns false.
-      return currentReader.getCurrent();
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() throws NoSuchElementException {
-      // A NoSuchElement will be thrown by the last FileBasedReader if getCurrentTimestamp()
-      // is called after advance() returns false.
-      return currentReader.getCurrentTimestamp();
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Close all readers that may have not yet been closed.
-      // If this reader has not been started, currentReader is null.
-      if (currentReader != null) {
-        currentReader.close();
-      }
-      while (fileReadersIterator.hasNext()) {
-        fileReadersIterator.next().close();
-      }
-    }
-
-    @Override
-    public FileBasedSource<T> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public FileBasedSource<T> splitAtFraction(double fraction) {
-      // Unsupported. TODO: implement.
-      LOG.debug("Dynamic splitting of FilePatternReader is unsupported.");
-      return null;
-    }
-
-    @Override
-    public Double getFractionConsumed() {
-      if (currentReader == null) {
-        return 0.0;
-      }
-      if (fileReaders.isEmpty()) {
-        return 1.0;
-      }
-      int index = fileReadersIterator.previousIndex();
-      int numReaders = fileReaders.size();
-      if (index == numReaders) {
-        return 1.0;
-      }
-      double before = 1.0 * index / numReaders;
-      double after = 1.0 * (index + 1) / numReaders;
-      Double fractionOfCurrentReader = currentReader.getFractionConsumed();
-      if (fractionOfCurrentReader == null) {
-        return before;
-      }
-      return before + fractionOfCurrentReader * (after - before);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java
deleted file mode 100644
index 288688e..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java
+++ /dev/null
@@ -1,329 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker;
-import com.google.cloud.dataflow.sdk.io.range.RangeTracker;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * A {@link BoundedSource} that uses offsets to define starting and ending positions.
- *
- * <p>{@link OffsetBasedSource} is a common base class for all bounded sources where the input can
- * be represented as a single range, and an input can be efficiently processed in parallel by
- * splitting the range into a set of disjoint ranges whose union is the original range. This class
- * should be used for sources that can be cheaply read starting at any given offset.
- * {@link OffsetBasedSource} stores the range and implements splitting into bundles.
- *
- * <p>Extend {@link OffsetBasedSource} to implement your own offset-based custom source.
- * {@link FileBasedSource}, which is a subclass of this, adds additional functionality useful for
- * custom sources that are based on files. If possible implementors should start from
- * {@link FileBasedSource} instead of {@link OffsetBasedSource}.
- *
- * <p>Consult {@link RangeTracker} for important semantics common to all sources defined by a range
- * of positions of a certain type, including the semantics of split points
- * ({@link OffsetBasedReader#isAtSplitPoint}).
- *
- * @param <T> Type of records represented by the source.
- * @see BoundedSource
- * @see FileBasedSource
- * @see RangeTracker
- */
-public abstract class OffsetBasedSource<T> extends BoundedSource<T> {
-  private final long startOffset;
-  private final long endOffset;
-  private final long minBundleSize;
-
-  /**
-   * @param startOffset starting offset (inclusive) of the source. Must be non-negative.
-   *
-   * @param endOffset ending offset (exclusive) of the source. Use {@link Long#MAX_VALUE} to
-   *        indicate that the entire source after {@code startOffset} should be read. Must be
-   *        {@code > startOffset}.
-   *
-   * @param minBundleSize minimum bundle size in offset units that should be used when splitting the
-   *                      source into sub-sources. This value may not be respected if the total
-   *                      range of the source is smaller than the specified {@code minBundleSize}.
-   *                      Must be non-negative.
-   */
-  public OffsetBasedSource(long startOffset, long endOffset, long minBundleSize) {
-    this.startOffset = startOffset;
-    this.endOffset = endOffset;
-    this.minBundleSize = minBundleSize;
-  }
-
-  /**
-   * Returns the starting offset of the source.
-   */
-  public long getStartOffset() {
-    return startOffset;
-  }
-
-  /**
-   * Returns the specified ending offset of the source. Any returned value greater than or equal to
-   * {@link #getMaxEndOffset(PipelineOptions)} should be treated as
-   * {@link #getMaxEndOffset(PipelineOptions)}.
-   */
-  public long getEndOffset() {
-    return endOffset;
-  }
-
-  /**
-   * Returns the minimum bundle size that should be used when splitting the source into sub-sources.
-   * This value may not be respected if the total range of the source is smaller than the specified
-   * {@code minBundleSize}.
-   */
-  public long getMinBundleSize() {
-    return minBundleSize;
-  }
-
-  @Override
-  public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
-    long trueEndOffset = (endOffset == Long.MAX_VALUE) ? getMaxEndOffset(options) : endOffset;
-    return getBytesPerOffset() * (trueEndOffset - getStartOffset());
-  }
-
-  @Override
-  public List<? extends OffsetBasedSource<T>> splitIntoBundles(
-      long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
-    // Split the range into bundles based on the desiredBundleSizeBytes. Final bundle is adjusted to
-    // make sure that we do not end up with a too small bundle at the end. If the desired bundle
-    // size is smaller than the minBundleSize of the source then minBundleSize will be used instead.
-
-    long desiredBundleSizeOffsetUnits = Math.max(
-        Math.max(1, desiredBundleSizeBytes / getBytesPerOffset()),
-        minBundleSize);
-
-    List<OffsetBasedSource<T>> subSources = new ArrayList<>();
-    long start = startOffset;
-    long maxEnd = Math.min(endOffset, getMaxEndOffset(options));
-
-    while (start < maxEnd) {
-      long end = start + desiredBundleSizeOffsetUnits;
-      end = Math.min(end, maxEnd);
-      // Avoid having a too small bundle at the end and ensure that we respect minBundleSize.
-      long remaining = maxEnd - end;
-      if ((remaining < desiredBundleSizeOffsetUnits / 4) || (remaining < minBundleSize)) {
-        end = maxEnd;
-      }
-      subSources.add(createSourceForSubrange(start, end));
-
-      start = end;
-    }
-    return subSources;
-  }
-
-  @Override
-  public void validate() {
-    Preconditions.checkArgument(
-        this.startOffset >= 0,
-        "Start offset has value %s, must be non-negative", this.startOffset);
-    Preconditions.checkArgument(
-        this.endOffset >= 0,
-        "End offset has value %s, must be non-negative", this.endOffset);
-    Preconditions.checkArgument(
-        this.startOffset < this.endOffset,
-        "Start offset %s must be before end offset %s",
-        this.startOffset, this.endOffset);
-    Preconditions.checkArgument(
-        this.minBundleSize >= 0,
-        "minBundleSize has value %s, must be non-negative",
-        this.minBundleSize);
-  }
-
-  @Override
-  public String toString() {
-    return "[" + startOffset + ", " + endOffset + ")";
-  }
-
-  /**
-   * Returns approximately how many bytes of data correspond to a single offset in this source.
-   * Used for translation between this source's range and methods defined in terms of bytes, such
-   * as {@link #getEstimatedSizeBytes} and {@link #splitIntoBundles}.
-   *
-   * <p>Defaults to {@code 1} byte, which is the common case for, e.g., file sources.
-   */
-  public long getBytesPerOffset() {
-    return 1L;
-  }
-
-  /**
-   * Returns the actual ending offset of the current source. The value returned by this function
-   * will be used to clip the end of the range {@code [startOffset, endOffset)} such that the
-   * range used is {@code [startOffset, min(endOffset, maxEndOffset))}.
-   *
-   * <p>As an example in which {@link OffsetBasedSource} is used to implement a file source, suppose
-   * that this source was constructed with an {@code endOffset} of {@link Long#MAX_VALUE} to
-   * indicate that a file should be read to the end. Then {@link #getMaxEndOffset} should determine
-   * the actual, exact size of the file in bytes and return it.
-   */
-  public abstract long getMaxEndOffset(PipelineOptions options) throws Exception;
-
-  /**
-   * Returns an {@link OffsetBasedSource} for a subrange of the current source. The
-   * subrange {@code [start, end)} must be within the range {@code [startOffset, endOffset)} of
-   * the current source, i.e. {@code startOffset <= start < end <= endOffset}.
-   */
-  public abstract OffsetBasedSource<T> createSourceForSubrange(long start, long end);
-
-  /**
-   * Whether this source should allow dynamic splitting of the offset ranges.
-   *
-   * <p>True by default. Override this to return false if the source cannot
-   * support dynamic splitting correctly. If this returns false,
-   * {@link OffsetBasedSource.OffsetBasedReader#splitAtFraction} will refuse all split requests.
-   */
-  public boolean allowsDynamicSplitting() {
-    return true;
-  }
-
-  /**
-   * A {@link Source.Reader} that implements code common to readers of all
-   * {@link OffsetBasedSource}s.
-   *
-   * <p>Subclasses have to implement:
-   * <ul>
-   *   <li>The methods {@link #startImpl} and {@link #advanceImpl} for reading the
-   *   first or subsequent records.
-   *   <li>The methods {@link #getCurrent}, {@link #getCurrentOffset}, and optionally
-   *   {@link #isAtSplitPoint} and {@link #getCurrentTimestamp} to access properties of
-   *   the last record successfully read by {@link #startImpl} or {@link #advanceImpl}.
-   * </ul>
-   */
-  public abstract static class OffsetBasedReader<T> extends BoundedReader<T> {
-    private static final Logger LOG = LoggerFactory.getLogger(OffsetBasedReader.class);
-
-    private OffsetBasedSource<T> source;
-
-    /** The {@link OffsetRangeTracker} managing the range and current position of the source. */
-    private final OffsetRangeTracker rangeTracker;
-
-    /**
-     * @param source the {@link OffsetBasedSource} to be read by the current reader.
-     */
-    public OffsetBasedReader(OffsetBasedSource<T> source) {
-      this.source = source;
-      this.rangeTracker = new OffsetRangeTracker(source.getStartOffset(), source.getEndOffset());
-    }
-
-    /**
-     * Returns the <i>starting</i> offset of the {@link Source.Reader#getCurrent current record},
-     * which has been read by the last successful {@link Source.Reader#start} or
-     * {@link Source.Reader#advance} call.
-     * <p>If no such call has been made yet, the return value is unspecified.
-     * <p>See {@link RangeTracker} for description of offset semantics.
-     */
-    protected abstract long getCurrentOffset() throws NoSuchElementException;
-
-    /**
-     * Returns whether the current record is at a split point (i.e., whether the current record
-     * would be the first record to be read by a source with a specified start offset of
-     * {@link #getCurrentOffset}).
-     *
-     * <p>See detailed documentation about split points in {@link RangeTracker}.
-     */
-    protected boolean isAtSplitPoint() throws NoSuchElementException {
-      return true;
-    }
-
-    @Override
-    public final boolean start() throws IOException {
-      return startImpl() && rangeTracker.tryReturnRecordAt(isAtSplitPoint(), getCurrentOffset());
-    }
-
-    @Override
-    public final boolean advance() throws IOException {
-      return advanceImpl() && rangeTracker.tryReturnRecordAt(isAtSplitPoint(), getCurrentOffset());
-    }
-
-    /**
-     * Initializes the {@link OffsetBasedSource.OffsetBasedReader} and advances to the first record,
-     * returning {@code true} if there is a record available to be read. This method will be
-     * invoked exactly once and may perform expensive setup operations that are needed to
-     * initialize the reader.
-     *
-     * <p>This function is the {@code OffsetBasedReader} implementation of
-     * {@link BoundedReader#start}. The key difference is that the implementor can ignore the
-     * possibility that it should no longer produce the first record, either because it has exceeded
-     * the original {@code endOffset} assigned to the reader, or because a concurrent call to
-     * {@link #splitAtFraction} has changed the source to shrink the offset range being read.
-     *
-     * @see BoundedReader#start
-     */
-    protected abstract boolean startImpl() throws IOException;
-
-    /**
-     * Advances to the next record and returns {@code true}, or returns false if there is no next
-     * record.
-     *
-     * <p>This function is the {@code OffsetBasedReader} implementation of
-     * {@link BoundedReader#advance}. The key difference is that the implementor can ignore the
-     * possibility that it should no longer produce the next record, either because it has exceeded
-     * the original {@code endOffset} assigned to the reader, or because a concurrent call to
-     * {@link #splitAtFraction} has changed the source to shrink the offset range being read.
-     *
-     * @see BoundedReader#advance
-     */
-    protected abstract boolean advanceImpl() throws IOException;
-
-    @Override
-    public synchronized OffsetBasedSource<T> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public Double getFractionConsumed() {
-      return rangeTracker.getFractionConsumed();
-    }
-
-    @Override
-    public final synchronized OffsetBasedSource<T> splitAtFraction(double fraction) {
-      if (!getCurrentSource().allowsDynamicSplitting()) {
-        return null;
-      }
-      if (rangeTracker.getStopPosition() == Long.MAX_VALUE) {
-        LOG.debug(
-            "Refusing to split unbounded OffsetBasedReader {} at fraction {}",
-            rangeTracker, fraction);
-        return null;
-      }
-      long splitOffset = rangeTracker.getPositionForFractionConsumed(fraction);
-      LOG.debug(
-          "Proposing to split OffsetBasedReader {} at fraction {} (offset {})",
-          rangeTracker, fraction, splitOffset);
-      if (!rangeTracker.trySplitAtPosition(splitOffset)) {
-        return null;
-      }
-      long start = source.getStartOffset();
-      long end = source.getEndOffset();
-      OffsetBasedSource<T> primary = source.createSourceForSubrange(start, splitOffset);
-      OffsetBasedSource<T> residual = source.createSourceForSubrange(splitOffset, end);
-      this.source = primary;
-      return residual;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java
deleted file mode 100644
index e5b8a39..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java
+++ /dev/null
@@ -1,322 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.api.client.repackaged.com.google.common.base.Preconditions;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * A helper interface for talking to Pubsub via an underlying transport.
- */
-public interface PubsubClient extends AutoCloseable {
-  /**
-   * Path representing a cloud project id.
-   */
-  class ProjectPath implements Serializable {
-    private final String path;
-
-    public ProjectPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      ProjectPath that = (ProjectPath) o;
-
-      return path.equals(that.path);
-
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static ProjectPath fromId(String projectId) {
-      return new ProjectPath(String.format("projects/%s", projectId));
-    }
-  }
-
-  /**
-   * Path representing a Pubsub subscription.
-   */
-  class SubscriptionPath implements Serializable {
-    private final String path;
-
-    public SubscriptionPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      Preconditions.checkState(splits.length == 4);
-      return String.format("/subscriptions/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      SubscriptionPath that = (SubscriptionPath) o;
-      return path.equals(that.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static SubscriptionPath fromName(String projectId, String subscriptionName) {
-      return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
-          projectId, subscriptionName));
-    }
-  }
-
-  /**
-   * Path representing a Pubsub topic.
-   */
-  class TopicPath implements Serializable {
-    private final String path;
-
-    public TopicPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      Preconditions.checkState(splits.length == 4);
-      return String.format("/topics/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      TopicPath topicPath = (TopicPath) o;
-      return path.equals(topicPath.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static TopicPath fromName(String projectId, String topicName) {
-      return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
-    }
-  }
-
-  /**
-   * A message to be sent to Pubsub.
-   */
-  class OutgoingMessage {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    /**
-     * Timestamp for element (ms since epoch).
-     */
-    public final long timestampMsSinceEpoch;
-
-    public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) {
-      this.elementBytes = elementBytes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-    }
-  }
-
-  /**
-   * A message received from Pubsub.
-   */
-  class IncomingMessage {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    /**
-     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
-     * or the custom timestamp associated with the message.
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * Timestamp (in system time) at which we requested the message (ms since epoch).
-     */
-    public final long requestTimeMsSinceEpoch;
-
-    /**
-     * Id to pass back to Pubsub to acknowledge receipt of this message.
-     */
-    public final String ackId;
-
-    /**
-     * Id to pass to the runner to distinguish this message from all others.
-     */
-    public final byte[] recordId;
-
-    public IncomingMessage(
-        byte[] elementBytes,
-        long timestampMsSinceEpoch,
-        long requestTimeMsSinceEpoch,
-        String ackId,
-        byte[] recordId) {
-      this.elementBytes = elementBytes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
-      this.ackId = ackId;
-      this.recordId = recordId;
-    }
-  }
-
-  /**
-   * Gracefully close the underlying transport.
-   */
-  @Override
-  void close();
-
-
-  /**
-   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
-   * published.
-   *
-   * @throws IOException
-   */
-  int publish(TopicPath topic, Iterable<OutgoingMessage> outgoingMessages) throws IOException;
-
-  /**
-   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
-   * Return the received messages, or empty collection if none were available. Does not
-   * wait for messages to arrive. Returned messages will record heir request time
-   * as {@code requestTimeMsSinceEpoch}.
-   *
-   * @throws IOException
-   */
-  Collection<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize)
-      throws IOException;
-
-  /**
-   * Acknowldege messages from {@code subscription} with {@code ackIds}.
-   *
-   * @throws IOException
-   */
-  void acknowledge(SubscriptionPath subscription, Iterable<String> ackIds) throws IOException;
-
-  /**
-   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
-   * be {@code deadlineSeconds} from now.
-   *
-   * @throws IOException
-   */
-  void modifyAckDeadline(
-      SubscriptionPath subscription, Iterable<String> ackIds,
-      int deadlineSeconds)
-      throws IOException;
-
-  /**
-   * Create {@code topic}.
-   *
-   * @throws IOException
-   */
-  void createTopic(TopicPath topic) throws IOException;
-
-  /*
-   * Delete {@code topic}.
-   *
-   * @throws IOException
-   */
-  void deleteTopic(TopicPath topic) throws IOException;
-
-  /**
-   * Return a list of topics for {@code project}.
-   *
-   * @throws IOException
-   */
-  Collection<TopicPath> listTopics(ProjectPath project) throws IOException;
-
-  /**
-   * Create {@code subscription} to {@code topic}.
-   *
-   * @throws IOException
-   */
-  void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException;
-
-  /**
-   * Delete {@code subscription}.
-   *
-   * @throws IOException
-   */
-  void deleteSubscription(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return a list of subscriptions for {@code topic} in {@code project}.
-   *
-   * @throws IOException
-   */
-  Collection<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java
deleted file mode 100644
index 6e34705..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java
+++ /dev/null
@@ -1,401 +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 com.google.cloud.dataflow.sdk.io;
-
-import com.google.api.client.util.DateTime;
-import com.google.auth.oauth2.GoogleCredentials;
-import com.google.cloud.dataflow.sdk.options.GcpOptions;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.hash.Hashing;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Timestamp;
-import com.google.pubsub.v1.AcknowledgeRequest;
-import com.google.pubsub.v1.DeleteSubscriptionRequest;
-import com.google.pubsub.v1.DeleteTopicRequest;
-import com.google.pubsub.v1.ListSubscriptionsRequest;
-import com.google.pubsub.v1.ListSubscriptionsResponse;
-import com.google.pubsub.v1.ListTopicsRequest;
-import com.google.pubsub.v1.ListTopicsResponse;
-import com.google.pubsub.v1.ModifyAckDeadlineRequest;
-import com.google.pubsub.v1.PublishRequest;
-import com.google.pubsub.v1.PublishResponse;
-import com.google.pubsub.v1.PublisherGrpc;
-import com.google.pubsub.v1.PubsubMessage;
-import com.google.pubsub.v1.PullRequest;
-import com.google.pubsub.v1.PullResponse;
-import com.google.pubsub.v1.ReceivedMessage;
-import com.google.pubsub.v1.SubscriberGrpc;
-import com.google.pubsub.v1.Subscription;
-import com.google.pubsub.v1.Topic;
-import io.grpc.Channel;
-import io.grpc.ClientInterceptors;
-import io.grpc.ManagedChannel;
-import io.grpc.auth.ClientAuthInterceptor;
-import io.grpc.netty.GrpcSslContexts;
-import io.grpc.netty.NegotiationType;
-import io.grpc.netty.NettyChannelBuilder;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A helper class for talking to Pubsub via grpc.
- */
-public class PubsubGrpcClient implements PubsubClient {
-  private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com";
-  private static final int PUBSUB_PORT = 443;
-  private static final List<String> PUBSUB_SCOPES =
-      Collections.singletonList("https://www.googleapis.com/auth/pubsub");
-  private static final int LIST_BATCH_SIZE = 1000;
-
-  /**
-   * Timeout for grpc calls (in s).
-   */
-  private static final int TIMEOUT_S = 15;
-
-  /**
-   * Underlying netty channel, or {@literal null} if closed.
-   */
-  @Nullable
-  private ManagedChannel publisherChannel;
-
-  /**
-   * Credentials determined from options and environment.
-   */
-  private final GoogleCredentials credentials;
-
-  /**
-   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
-   * instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
-   */
-  @Nullable
-  private final String idLabel;
-
-  /**
-   * Cached stubs, or null if not cached.
-   */
-  @Nullable
-  private PublisherGrpc.PublisherBlockingStub cachedPublisherStub;
-  private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub;
-
-  private PubsubGrpcClient(
-      @Nullable String timestampLabel, @Nullable String idLabel,
-      ManagedChannel publisherChannel, GoogleCredentials credentials) {
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.publisherChannel = publisherChannel;
-    this.credentials = credentials;
-  }
-
-  /**
-   * Construct a new Pubsub grpc client. It should be closed via {@link #close} in order
-   * to ensure tidy cleanup of underlying netty resources. (Or use the try-with-resources
-   * construct since this class is {@link AutoCloseable}). If non-{@literal null}, use
-   * {@code timestampLabel} and {@code idLabel} to store custom timestamps/ids within
-   * message metadata.
-   */
-  public static PubsubGrpcClient newClient(
-      @Nullable String timestampLabel, @Nullable String idLabel,
-      GcpOptions options) throws IOException {
-    ManagedChannel channel = NettyChannelBuilder
-        .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT)
-        .negotiationType(NegotiationType.TLS)
-        .sslContext(GrpcSslContexts.forClient().ciphers(null).build())
-        .build();
-    // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the
-    // various command line options. It currently only supports the older
-    // com.google.api.client.auth.oauth2.Credentials.
-    GoogleCredentials credentials = GoogleCredentials.getApplicationDefault();
-    return new PubsubGrpcClient(timestampLabel, idLabel, channel, credentials);
-  }
-
-  /**
-   * Gracefully close the underlying netty channel.
-   */
-  @Override
-  public void close() {
-    Preconditions.checkState(publisherChannel != null, "Client has already been closed");
-    publisherChannel.shutdown();
-    try {
-      publisherChannel.awaitTermination(TIMEOUT_S, TimeUnit.SECONDS);
-    } catch (InterruptedException e) {
-      // Ignore.
-      Thread.currentThread().interrupt();
-    }
-    publisherChannel = null;
-    cachedPublisherStub = null;
-    cachedSubscriberStub = null;
-  }
-
-  /**
-   * Return channel with interceptor for returning credentials.
-   */
-  private Channel newChannel() throws IOException {
-    Preconditions.checkState(publisherChannel != null, "PubsubGrpcClient has been closed");
-    ClientAuthInterceptor interceptor =
-        new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor());
-    return ClientInterceptors.intercept(publisherChannel, interceptor);
-  }
-
-  /**
-   * Return a stub for making a publish request with a timeout.
-   */
-  private PublisherGrpc.PublisherBlockingStub publisherStub() throws IOException {
-    if (cachedPublisherStub == null) {
-      cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel());
-    }
-    return cachedPublisherStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Return a stub for making a subscribe request with a timeout.
-   */
-  private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOException {
-    if (cachedSubscriberStub == null) {
-      cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel());
-    }
-    return cachedSubscriberStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS);
-  }
-
-  @Override
-  public int publish(TopicPath topic, Iterable<OutgoingMessage> outgoingMessages)
-      throws IOException {
-    PublishRequest.Builder request = PublishRequest.newBuilder()
-                                                   .setTopic(topic.getPath());
-    for (OutgoingMessage outgoingMessage : outgoingMessages) {
-      PubsubMessage.Builder message =
-          PubsubMessage.newBuilder()
-                       .setData(ByteString.copyFrom(outgoingMessage.elementBytes));
-
-      if (timestampLabel != null) {
-        message.getMutableAttributes()
-               .put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
-      }
-
-      if (idLabel != null) {
-        message.getMutableAttributes()
-               .put(idLabel,
-                   Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString());
-      }
-
-      request.addMessages(message);
-    }
-
-    PublishResponse response = publisherStub().publish(request.build());
-    return response.getMessageIdsCount();
-  }
-
-  @Override
-  public Collection<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize) throws IOException {
-    PullRequest request = PullRequest.newBuilder()
-                                     .setSubscription(subscription.getPath())
-                                     .setReturnImmediately(true)
-                                     .setMaxMessages(batchSize)
-                                     .build();
-    PullResponse response = subscriberStub().pull(request);
-    if (response.getReceivedMessagesCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessagesCount());
-    for (ReceivedMessage message : response.getReceivedMessagesList()) {
-      PubsubMessage pubsubMessage = message.getMessage();
-      Map<String, String> attributes = pubsubMessage.getAttributes();
-
-      // Payload.
-      byte[] elementBytes = pubsubMessage.getData().toByteArray();
-
-      // Timestamp.
-      // Start with Pubsub processing time.
-      Timestamp timestampProto = pubsubMessage.getPublishTime();
-      long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L;
-      if (timestampLabel != null && attributes != null) {
-        String timestampString = attributes.get(timestampLabel);
-        if (timestampString != null && !timestampString.isEmpty()) {
-          try {
-            // Try parsing as milliseconds since epoch. Note there is no way to parse a
-            // string in RFC 3339 format here.
-            // Expected IllegalArgumentException if parsing fails; we use that to fall back
-            // to RFC 3339.
-            timestampMsSinceEpoch = Long.parseLong(timestampString);
-          } catch (IllegalArgumentException e1) {
-            try {
-              // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an
-              // IllegalArgumentException if parsing fails, and the caller should handle.
-              timestampMsSinceEpoch = DateTime.parseRfc3339(timestampString).getValue();
-            } catch (IllegalArgumentException e2) {
-              // Fallback to Pubsub processing time.
-            }
-          }
-        }
-        // else: fallback to Pubsub processing time.
-      }
-      // else: fallback to Pubsub processing time.
-
-      // Ack id.
-      String ackId = message.getAckId();
-      Preconditions.checkState(ackId != null && !ackId.isEmpty());
-
-      // Record id, if any.
-      @Nullable byte[] recordId = null;
-      if (idLabel != null && attributes != null) {
-        String recordIdString = attributes.get(idLabel);
-        if (recordIdString != null && !recordIdString.isEmpty()) {
-          recordId = recordIdString.getBytes();
-        }
-      }
-      if (recordId == null) {
-        recordId = pubsubMessage.getMessageId().getBytes();
-      }
-
-      incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch,
-          requestTimeMsSinceEpoch, ackId, recordId));
-    }
-    return incomingMessages;
-  }
-
-  @Override
-  public void acknowledge(SubscriptionPath subscription, Iterable<String> ackIds)
-      throws IOException {
-    AcknowledgeRequest request = AcknowledgeRequest.newBuilder()
-                                                   .setSubscription(subscription.getPath())
-                                                   .addAllAckIds(ackIds)
-                                                   .build();
-    subscriberStub().acknowledge(request); // ignore Empty result.
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, Iterable<String> ackIds, int
-      deadlineSeconds)
-      throws IOException {
-    ModifyAckDeadlineRequest request =
-        ModifyAckDeadlineRequest.newBuilder()
-                                .setSubscription(subscription.getPath())
-                                .addAllAckIds(ackIds)
-                                .setAckDeadlineSeconds(deadlineSeconds)
-                                .build();
-    subscriberStub().modifyAckDeadline(request); // ignore Empty result.
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    Topic request = Topic.newBuilder()
-                         .setName(topic.getPath())
-                         .build();
-    publisherStub().createTopic(request); // ignore Topic result.
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    DeleteTopicRequest request = DeleteTopicRequest.newBuilder()
-                                                   .setTopic(topic.getPath())
-                                                   .build();
-    publisherStub().deleteTopic(request); // ignore Empty result.
-  }
-
-  @Override
-  public Collection<TopicPath> listTopics(ProjectPath project) throws IOException {
-    ListTopicsRequest.Builder request =
-        ListTopicsRequest.newBuilder()
-                         .setProject(project.getPath())
-                         .setPageSize(LIST_BATCH_SIZE);
-    ListTopicsResponse response = publisherStub().listTopics(request.build());
-    if (response.getTopicsCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<TopicPath> topics = new ArrayList<>(response.getTopicsCount());
-    while (true) {
-      for (Topic topic : response.getTopicsList()) {
-        topics.add(new TopicPath(topic.getName()));
-      }
-      if (response.getNextPageToken().isEmpty()) {
-        break;
-      }
-      request.setPageToken(response.getNextPageToken());
-      response = publisherStub().listTopics(request.build());
-    }
-    return topics;
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException {
-    Subscription request = Subscription.newBuilder()
-                                       .setTopic(topic.getPath())
-                                       .setName(subscription.getPath())
-                                       .setAckDeadlineSeconds(ackDeadlineSeconds)
-                                       .build();
-    subscriberStub().createSubscription(request); // ignore Subscription result.
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    DeleteSubscriptionRequest request =
-        DeleteSubscriptionRequest.newBuilder()
-                                 .setSubscription(subscription.getPath())
-                                 .build();
-    subscriberStub().deleteSubscription(request); // ignore Empty result.
-  }
-
-  @Override
-  public Collection<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException {
-    ListSubscriptionsRequest.Builder request =
-        ListSubscriptionsRequest.newBuilder()
-                                .setProject(project.getPath())
-                                .setPageSize(LIST_BATCH_SIZE);
-    ListSubscriptionsResponse response = subscriberStub().listSubscriptions(request.build());
-    if (response.getSubscriptionsCount() == 0) {
-      return ImmutableList.of();
-    }
-    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptionsCount());
-    while (true) {
-      for (Subscription subscription : response.getSubscriptionsList()) {
-        if (subscription.getTopic().equals(topic.getPath())) {
-          subscriptions.add(new SubscriptionPath(subscription.getName()));
-        }
-      }
-      if (response.getNextPageToken().isEmpty()) {
-        break;
-      }
-      request.setPageToken(response.getNextPageToken());
-      response = subscriberStub().listSubscriptions(request.build());
-    }
-    return subscriptions;
-  }
-}


[29/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java
deleted file mode 100644
index 764c0cb..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java
+++ /dev/null
@@ -1,605 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.hamcrest.Matchers.hasEntry;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.lessThanOrEqualTo;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.cloud.dataflow.sdk.PipelineResult.State;
-import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff;
-import com.google.cloud.dataflow.sdk.util.MonitoringUtil;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSetMultimap;
-
-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 java.io.IOException;
-import java.math.BigDecimal;
-import java.net.SocketTimeoutException;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for DataflowPipelineJob.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineJobTest {
-  private static final String PROJECT_ID = "someProject";
-  private static final String JOB_ID = "1234";
-
-  @Mock
-  private Dataflow mockWorkflowClient;
-  @Mock
-  private Dataflow.Projects mockProjects;
-  @Mock
-  private Dataflow.Projects.Jobs mockJobs;
-  @Rule
-  public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-
-    when(mockWorkflowClient.projects()).thenReturn(mockProjects);
-    when(mockProjects.jobs()).thenReturn(mockJobs);
-  }
-
-  /**
-   * Validates that a given time is valid for the total time slept by a
-   * AttemptBoundedExponentialBackOff given the number of retries and
-   * an initial polling interval.
-   *
-   * @param pollingIntervalMillis The initial polling interval given.
-   * @param attempts The number of attempts made
-   * @param timeSleptMillis The amount of time slept by the clock. This is checked
-   * against the valid interval.
-   */
-  void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) {
-    long highSum = 0;
-    long lowSum = 0;
-    for (int i = 1; i < attempts; i++) {
-      double currentInterval =
-          pollingIntervalMillis
-          * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1);
-      double offset =
-          AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval;
-      highSum += Math.round(currentInterval + offset);
-      lowSum += Math.round(currentInterval - offset);
-    }
-    assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum)));
-  }
-
-  @Test
-  public void testWaitToFinishMessagesFail() throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    Job statusResponse = new Job();
-    statusResponse.setCurrentState("JOB_STATE_" + State.DONE.name());
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenReturn(statusResponse);
-
-    MonitoringUtil.JobMessagesHandler jobHandler = mock(MonitoringUtil.JobMessagesHandler.class);
-    Dataflow.Projects.Jobs.Messages mockMessages =
-        mock(Dataflow.Projects.Jobs.Messages.class);
-    Messages.List listRequest = mock(Dataflow.Projects.Jobs.Messages.List.class);
-    when(mockJobs.messages()).thenReturn(mockMessages);
-    when(mockMessages.list(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(listRequest);
-    when(listRequest.execute()).thenThrow(SocketTimeoutException.class);
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
-    State state = job.waitToFinish(5, TimeUnit.MINUTES, jobHandler, fastClock, fastClock);
-    assertEquals(null, state);
-  }
-
-  public State mockWaitToFinishInState(State state) throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    Job statusResponse = new Job();
-    statusResponse.setCurrentState("JOB_STATE_" + state.name());
-
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenReturn(statusResponse);
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
-    return job.waitToFinish(1, TimeUnit.MINUTES, null, fastClock, fastClock);
-  }
-
-  /**
-   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#DONE DONE}
-   * state is terminal.
-   */
-  @Test
-  public void testWaitToFinishDone() throws Exception {
-    assertEquals(State.DONE, mockWaitToFinishInState(State.DONE));
-  }
-
-  /**
-   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
-   * state is terminal.
-   */
-  @Test
-  public void testWaitToFinishFailed() throws Exception {
-    assertEquals(State.FAILED, mockWaitToFinishInState(State.FAILED));
-  }
-
-  /**
-   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
-   * state is terminal.
-   */
-  @Test
-  public void testWaitToFinishCancelled() throws Exception {
-    assertEquals(State.CANCELLED, mockWaitToFinishInState(State.CANCELLED));
-  }
-
-  /**
-   * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
-   * state is terminal.
-   */
-  @Test
-  public void testWaitToFinishUpdated() throws Exception {
-    assertEquals(State.UPDATED, mockWaitToFinishInState(State.UPDATED));
-  }
-
-  @Test
-  public void testWaitToFinishFail() throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenThrow(IOException.class);
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
-    long startTime = fastClock.nanoTime();
-    State state = job.waitToFinish(5, TimeUnit.MINUTES, null, fastClock, fastClock);
-    assertEquals(null, state);
-    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
-    checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL,
-        DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff);
-  }
-
-  @Test
-  public void testWaitToFinishTimeFail() throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenThrow(IOException.class);
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-    long startTime = fastClock.nanoTime();
-    State state = job.waitToFinish(4, TimeUnit.MILLISECONDS, null, fastClock, fastClock);
-    assertEquals(null, state);
-    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
-    // Should only sleep for the 4 ms remaining.
-    assertEquals(timeDiff, 4L);
-  }
-
-  @Test
-  public void testGetStateReturnsServiceState() throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    Job statusResponse = new Job();
-    statusResponse.setCurrentState("JOB_STATE_" + State.RUNNING.name());
-
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenReturn(statusResponse);
-
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
-    assertEquals(
-        State.RUNNING,
-        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
-  }
-
-  @Test
-  public void testGetStateWithExceptionReturnsUnknown() throws Exception {
-    Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
-    when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
-    when(statusRequest.execute()).thenThrow(IOException.class);
-    DataflowAggregatorTransforms dataflowAggregatorTransforms =
-        mock(DataflowAggregatorTransforms.class);
-
-    DataflowPipelineJob job = new DataflowPipelineJob(
-        PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
-    long startTime = fastClock.nanoTime();
-    assertEquals(
-        State.UNKNOWN,
-        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
-    long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
-    checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL,
-        DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff);
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithNoMetricUpdatesReturnsEmptyValue()
-      throws IOException, AggregatorRetrievalException {
-    Aggregator<?, ?> aggregator = mock(Aggregator.class);
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    JobMetrics jobMetrics = new JobMetrics();
-    when(getMetrics.execute()).thenReturn(jobMetrics);
-
-    jobMetrics.setMetrics(ImmutableList.<MetricUpdate>of());
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    AggregatorValues<?> values = job.getAggregatorValues(aggregator);
-
-    assertThat(values.getValues(), empty());
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithNullMetricUpdatesReturnsEmptyValue()
-      throws IOException, AggregatorRetrievalException {
-    Aggregator<?, ?> aggregator = mock(Aggregator.class);
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    JobMetrics jobMetrics = new JobMetrics();
-    when(getMetrics.execute()).thenReturn(jobMetrics);
-
-    jobMetrics.setMetrics(null);
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    AggregatorValues<?> values = job.getAggregatorValues(aggregator);
-
-    assertThat(values.getValues(), empty());
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithSingleMetricUpdateReturnsSingletonCollection()
-      throws IOException, AggregatorRetrievalException {
-    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
-    String aggregatorName = "agg";
-    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    JobMetrics jobMetrics = new JobMetrics();
-    when(getMetrics.execute()).thenReturn(jobMetrics);
-
-    MetricUpdate update = new MetricUpdate();
-    long stepValue = 1234L;
-    update.setScalar(new BigDecimal(stepValue));
-
-    MetricStructuredName structuredName = new MetricStructuredName();
-    structuredName.setName(aggregatorName);
-    structuredName.setContext(ImmutableMap.of("step", stepName));
-    update.setName(structuredName);
-
-    jobMetrics.setMetrics(ImmutableList.of(update));
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
-    assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
-    assertThat(values.getValuesAtSteps().size(), equalTo(1));
-    assertThat(values.getValues(), contains(stepValue));
-    assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue)));
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithMultipleMetricUpdatesReturnsCollection()
-      throws IOException, AggregatorRetrievalException {
-    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
-    String aggregatorName = "agg";
-    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
-
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> otherTransform = mock(PTransform.class);
-    String otherStepName = "s88";
-    String otherFullName = "Spam/Ham/Eggs";
-    AppliedPTransform<?, ?, ?> otherAppliedTransform =
-        appliedPTransform(otherFullName, otherTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(
-                                aggregator, pTransform, aggregator, otherTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(
-            appliedTransform, stepName, otherAppliedTransform, otherStepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    JobMetrics jobMetrics = new JobMetrics();
-    when(getMetrics.execute()).thenReturn(jobMetrics);
-
-    MetricUpdate updateOne = new MetricUpdate();
-    long stepValue = 1234L;
-    updateOne.setScalar(new BigDecimal(stepValue));
-
-    MetricStructuredName structuredNameOne = new MetricStructuredName();
-    structuredNameOne.setName(aggregatorName);
-    structuredNameOne.setContext(ImmutableMap.of("step", stepName));
-    updateOne.setName(structuredNameOne);
-
-    MetricUpdate updateTwo = new MetricUpdate();
-    long stepValueTwo = 1024L;
-    updateTwo.setScalar(new BigDecimal(stepValueTwo));
-
-    MetricStructuredName structuredNameTwo = new MetricStructuredName();
-    structuredNameTwo.setName(aggregatorName);
-    structuredNameTwo.setContext(ImmutableMap.of("step", otherStepName));
-    updateTwo.setName(structuredNameTwo);
-
-    jobMetrics.setMetrics(ImmutableList.of(updateOne, updateTwo));
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
-    assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
-    assertThat(values.getValuesAtSteps(), hasEntry(otherFullName, stepValueTwo));
-    assertThat(values.getValuesAtSteps().size(), equalTo(2));
-    assertThat(values.getValues(), containsInAnyOrder(stepValue, stepValueTwo));
-    assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue + stepValueTwo)));
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithUnrelatedMetricUpdateIgnoresUpdate()
-      throws IOException, AggregatorRetrievalException {
-    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
-    String aggregatorName = "agg";
-    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    JobMetrics jobMetrics = new JobMetrics();
-    when(getMetrics.execute()).thenReturn(jobMetrics);
-
-    MetricUpdate ignoredUpdate = new MetricUpdate();
-    ignoredUpdate.setScalar(null);
-
-    MetricStructuredName ignoredName = new MetricStructuredName();
-    ignoredName.setName("ignoredAggregator.elementCount.out0");
-    ignoredName.setContext(null);
-    ignoredUpdate.setName(ignoredName);
-
-    jobMetrics.setMetrics(ImmutableList.of(ignoredUpdate));
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
-    assertThat(values.getValuesAtSteps().entrySet(), empty());
-    assertThat(values.getValues(), empty());
-  }
-
-  @Test
-  public void testGetAggregatorValuesWithUnusedAggregatorThrowsException()
-      throws AggregatorRetrievalException {
-    Aggregator<?, ?> aggregator = mock(Aggregator.class);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of().asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("not used in this pipeline");
-
-    job.getAggregatorValues(aggregator);
-  }
-
-  @Test
-  public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetrievalException()
-      throws IOException, AggregatorRetrievalException {
-    CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
-    String aggregatorName = "agg";
-    Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
-    @SuppressWarnings("unchecked")
-    PTransform<PInput, POutput> pTransform = mock(PTransform.class);
-    String stepName = "s1";
-    String fullName = "Foo/Bar/Baz";
-    AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
-    DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
-        ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
-        ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
-    GetMetrics getMetrics = mock(GetMetrics.class);
-    when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
-    IOException cause = new IOException();
-    when(getMetrics.execute()).thenThrow(cause);
-
-    Get getState = mock(Get.class);
-    when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
-    Job modelJob = new Job();
-    when(getState.execute()).thenReturn(modelJob);
-    modelJob.setCurrentState(State.RUNNING.toString());
-
-    DataflowPipelineJob job =
-        new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
-    thrown.expect(AggregatorRetrievalException.class);
-    thrown.expectCause(is(cause));
-    thrown.expectMessage(aggregator.toString());
-    thrown.expectMessage("when retrieving Aggregator values for");
-
-    job.getAggregatorValues(aggregator);
-  }
-
-  private static class TestAggregator<InT, OutT> implements Aggregator<InT, OutT> {
-    private final CombineFn<InT, ?, OutT> combineFn;
-    private final String name;
-
-    public TestAggregator(CombineFn<InT, ?, OutT> combineFn, String name) {
-      this.combineFn = combineFn;
-      this.name = name;
-    }
-
-    @Override
-    public void addValue(InT value) {
-      throw new AssertionError();
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public CombineFn<InT, ?, OutT> getCombineFn() {
-      return combineFn;
-    }
-  }
-
-  private AppliedPTransform<?, ?, ?> appliedPTransform(
-      String fullName, PTransform<PInput, POutput> transform) {
-    return AppliedPTransform.of(fullName, mock(PInput.class), mock(POutput.class), transform);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java
deleted file mode 100644
index 4850939..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java
+++ /dev/null
@@ -1,73 +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 com.google.cloud.dataflow.sdk.runners;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ServiceLoader;
-
-/** Tests for {@link DataflowPipelineRegistrar}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineRegistrarTest {
-  @Test
-  public void testCorrectOptionsAreReturned() {
-    assertEquals(ImmutableList.of(DataflowPipelineOptions.class,
-                                  BlockingDataflowPipelineOptions.class),
-        new DataflowPipelineRegistrar.Options().getPipelineOptions());
-  }
-
-  @Test
-  public void testCorrectRunnersAreReturned() {
-    assertEquals(ImmutableList.of(DataflowPipelineRunner.class,
-                                  BlockingDataflowPipelineRunner.class),
-        new DataflowPipelineRegistrar.Runner().getPipelineRunners());
-  }
-
-  @Test
-  public void testServiceLoaderForOptions() {
-    for (PipelineOptionsRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
-      if (registrar instanceof DataflowPipelineRegistrar.Options) {
-        return;
-      }
-    }
-    fail("Expected to find " + DataflowPipelineRegistrar.Options.class);
-  }
-
-  @Test
-  public void testServiceLoaderForRunner() {
-    for (PipelineRunnerRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
-      if (registrar instanceof DataflowPipelineRegistrar.Runner) {
-        return;
-      }
-    }
-    fail("Expected to find " + DataflowPipelineRegistrar.Runner.class);
-  }
-}