You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ie...@apache.org on 2017/04/19 13:09:10 UTC

[01/18] beam git commit: [BEAM-1994] Remove Flink examples package

Repository: beam
Updated Branches:
  refs/heads/master 8a00f2254 -> 83193698d


http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
new file mode 100644
index 0000000..90f95d6
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
@@ -0,0 +1,464 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.util.InstantiationUtil;
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Matchers;
+
+/**
+ * Tests for {@link UnboundedSourceWrapper}.
+ */
+@RunWith(Enclosed.class)
+public class UnboundedSourceWrapperTest {
+
+  /**
+   * Parameterized tests.
+   */
+  @RunWith(Parameterized.class)
+  public static class UnboundedSourceWrapperTestWithParams {
+    private final int numTasks;
+    private final int numSplits;
+
+    public UnboundedSourceWrapperTestWithParams(int numTasks, int numSplits) {
+      this.numTasks = numTasks;
+      this.numSplits = numSplits;
+    }
+
+    @Parameterized.Parameters
+    public static Collection<Object[]> data() {
+      /*
+       * Parameters for initializing the tests:
+       * {numTasks, numSplits}
+       * The test currently assumes powers of two for some assertions.
+       */
+      return Arrays.asList(new Object[][]{
+          {1, 1}, {1, 2}, {1, 4},
+          {2, 1}, {2, 2}, {2, 4},
+          {4, 1}, {4, 2}, {4, 4}
+      });
+    }
+
+    /**
+     * Creates a {@link UnboundedSourceWrapper} that has one or multiple readers per source.
+     * If numSplits > numTasks the source has one source will manage multiple readers.
+     */
+    @Test
+    public void testReaders() throws Exception {
+      final int numElements = 20;
+      final Object checkpointLock = new Object();
+      PipelineOptions options = PipelineOptionsFactory.create();
+
+      // this source will emit exactly NUM_ELEMENTS across all parallel readers,
+      // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
+      // elements later.
+      TestCountingSource source = new TestCountingSource(numElements);
+      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
+          new UnboundedSourceWrapper<>(options, source, numSplits);
+
+      assertEquals(numSplits, flinkWrapper.getSplitSources().size());
+
+      StreamSource<WindowedValue<
+          KV<Integer, Integer>>,
+          UnboundedSourceWrapper<
+              KV<Integer, Integer>,
+              TestCountingSource.CounterMark>> sourceOperator = new StreamSource<>(flinkWrapper);
+
+      setupSourceOperator(sourceOperator, numTasks);
+
+      try {
+        sourceOperator.open();
+        sourceOperator.run(checkpointLock,
+            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
+              private int count = 0;
+
+              @Override
+              public void emitWatermark(Watermark watermark) {
+              }
+
+              @Override
+              public void emitLatencyMarker(LatencyMarker latencyMarker) {
+              }
+
+              @Override
+              public void collect(
+                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
+
+                count++;
+                if (count >= numElements) {
+                  throw new SuccessException();
+                }
+              }
+
+              @Override
+              public void close() {
+
+              }
+            });
+      } catch (SuccessException e) {
+
+        assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
+
+        // success
+        return;
+      }
+      fail("Read terminated without producing expected number of outputs");
+    }
+
+    /**
+     * Verify that snapshot/restore work as expected. We bring up a source and cancel
+     * after seeing a certain number of elements. Then we snapshot that source,
+     * bring up a completely new source that we restore from the snapshot and verify
+     * that we see all expected elements in the end.
+     */
+    @Test
+    public void testRestore() throws Exception {
+      final int numElements = 20;
+      final Object checkpointLock = new Object();
+      PipelineOptions options = PipelineOptionsFactory.create();
+
+      // this source will emit exactly NUM_ELEMENTS across all parallel readers,
+      // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
+      // elements later.
+      TestCountingSource source = new TestCountingSource(numElements);
+      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
+          new UnboundedSourceWrapper<>(options, source, numSplits);
+
+      assertEquals(numSplits, flinkWrapper.getSplitSources().size());
+
+      StreamSource<
+          WindowedValue<KV<Integer, Integer>>,
+          UnboundedSourceWrapper<
+              KV<Integer, Integer>,
+              TestCountingSource.CounterMark>> sourceOperator = new StreamSource<>(flinkWrapper);
+
+
+      OperatorStateStore backend = mock(OperatorStateStore.class);
+
+      TestingListState<KV<UnboundedSource, TestCountingSource.CounterMark>>
+          listState = new TestingListState<>();
+
+      when(backend.getOperatorState(Matchers.any(ListStateDescriptor.class)))
+          .thenReturn(listState);
+
+      StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+      when(initializationContext.getOperatorStateStore()).thenReturn(backend);
+      when(initializationContext.isRestored()).thenReturn(false, true);
+
+      flinkWrapper.initializeState(initializationContext);
+
+      setupSourceOperator(sourceOperator, numTasks);
+
+      final Set<KV<Integer, Integer>> emittedElements = new HashSet<>();
+
+      boolean readFirstBatchOfElements = false;
+
+      try {
+        sourceOperator.open();
+        sourceOperator.run(checkpointLock,
+            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
+              private int count = 0;
+
+              @Override
+              public void emitWatermark(Watermark watermark) {
+              }
+
+              @Override
+              public void emitLatencyMarker(LatencyMarker latencyMarker) {
+              }
+
+              @Override
+              public void collect(
+                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
+
+                emittedElements.add(windowedValueStreamRecord.getValue().getValue());
+                count++;
+                if (count >= numElements / 2) {
+                  throw new SuccessException();
+                }
+              }
+
+              @Override
+              public void close() {
+
+              }
+            });
+      } catch (SuccessException e) {
+        // success
+        readFirstBatchOfElements = true;
+      }
+
+      assertTrue("Did not successfully read first batch of elements.", readFirstBatchOfElements);
+
+      // draw a snapshot
+      flinkWrapper.snapshotState(new StateSnapshotContextSynchronousImpl(0, 0));
+
+      // test snapshot offsets
+      assertEquals(flinkWrapper.getLocalSplitSources().size(),
+          listState.getList().size());
+      int totalEmit = 0;
+      for (KV<UnboundedSource, TestCountingSource.CounterMark> kv : listState.get()) {
+        totalEmit += kv.getValue().current + 1;
+      }
+      assertEquals(numElements / 2, totalEmit);
+
+      // test that finalizeCheckpoint on CheckpointMark is called
+      final ArrayList<Integer> finalizeList = new ArrayList<>();
+      TestCountingSource.setFinalizeTracker(finalizeList);
+      flinkWrapper.notifyCheckpointComplete(0);
+      assertEquals(flinkWrapper.getLocalSplitSources().size(), finalizeList.size());
+
+      // create a completely new source but restore from the snapshot
+      TestCountingSource restoredSource = new TestCountingSource(numElements);
+      UnboundedSourceWrapper<
+          KV<Integer, Integer>, TestCountingSource.CounterMark> restoredFlinkWrapper =
+          new UnboundedSourceWrapper<>(options, restoredSource, numSplits);
+
+      assertEquals(numSplits, restoredFlinkWrapper.getSplitSources().size());
+
+      StreamSource<
+          WindowedValue<KV<Integer, Integer>>,
+          UnboundedSourceWrapper<
+              KV<Integer, Integer>,
+              TestCountingSource.CounterMark>> restoredSourceOperator =
+          new StreamSource<>(restoredFlinkWrapper);
+
+      setupSourceOperator(restoredSourceOperator, numTasks);
+
+      // restore snapshot
+      restoredFlinkWrapper.initializeState(initializationContext);
+
+      boolean readSecondBatchOfElements = false;
+
+      // run again and verify that we see the other elements
+      try {
+        restoredSourceOperator.open();
+        restoredSourceOperator.run(checkpointLock,
+            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
+              private int count = 0;
+
+              @Override
+              public void emitWatermark(Watermark watermark) {
+              }
+
+              @Override
+              public void emitLatencyMarker(LatencyMarker latencyMarker) {
+              }
+
+              @Override
+              public void collect(
+                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
+                emittedElements.add(windowedValueStreamRecord.getValue().getValue());
+                count++;
+                if (count >= numElements / 2) {
+                  throw new SuccessException();
+                }
+              }
+
+              @Override
+              public void close() {
+
+              }
+            });
+      } catch (SuccessException e) {
+        // success
+        readSecondBatchOfElements = true;
+      }
+
+      assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
+
+      assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements);
+
+      // verify that we saw all NUM_ELEMENTS elements
+      assertTrue(emittedElements.size() == numElements);
+    }
+
+    @Test
+    public void testNullCheckpoint() throws Exception {
+      final int numElements = 20;
+      PipelineOptions options = PipelineOptionsFactory.create();
+
+      TestCountingSource source = new TestCountingSource(numElements) {
+        @Override
+        public Coder<CounterMark> getCheckpointMarkCoder() {
+          return null;
+        }
+      };
+      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
+          new UnboundedSourceWrapper<>(options, source, numSplits);
+
+      OperatorStateStore backend = mock(OperatorStateStore.class);
+
+      TestingListState<KV<UnboundedSource, TestCountingSource.CounterMark>>
+          listState = new TestingListState<>();
+
+      when(backend.getOperatorState(Matchers.any(ListStateDescriptor.class)))
+          .thenReturn(listState);
+
+      StateInitializationContext initializationContext = mock(StateInitializationContext.class);
+
+      when(initializationContext.getOperatorStateStore()).thenReturn(backend);
+      when(initializationContext.isRestored()).thenReturn(false, true);
+
+      flinkWrapper.initializeState(initializationContext);
+
+      StreamSource sourceOperator = new StreamSource<>(flinkWrapper);
+      setupSourceOperator(sourceOperator, numTasks);
+      sourceOperator.open();
+
+      flinkWrapper.snapshotState(new StateSnapshotContextSynchronousImpl(0, 0));
+
+      assertEquals(0, listState.getList().size());
+
+      UnboundedSourceWrapper<
+          KV<Integer, Integer>, TestCountingSource.CounterMark> restoredFlinkWrapper =
+          new UnboundedSourceWrapper<>(options, new TestCountingSource(numElements),
+              numSplits);
+
+      StreamSource restoredSourceOperator = new StreamSource<>(flinkWrapper);
+      setupSourceOperator(restoredSourceOperator, numTasks);
+      sourceOperator.open();
+
+      restoredFlinkWrapper.initializeState(initializationContext);
+
+      assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
+
+    }
+
+    @SuppressWarnings("unchecked")
+    private static <T> void setupSourceOperator(StreamSource<T, ?> operator, int numSubTasks) {
+      ExecutionConfig executionConfig = new ExecutionConfig();
+      StreamConfig cfg = new StreamConfig(new Configuration());
+
+      cfg.setTimeCharacteristic(TimeCharacteristic.EventTime);
+
+      Environment env = new DummyEnvironment("MockTwoInputTask", numSubTasks, 0);
+
+      StreamTask<?, ?> mockTask = mock(StreamTask.class);
+      when(mockTask.getName()).thenReturn("Mock Task");
+      when(mockTask.getCheckpointLock()).thenReturn(new Object());
+      when(mockTask.getConfiguration()).thenReturn(cfg);
+      when(mockTask.getEnvironment()).thenReturn(env);
+      when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
+      when(mockTask.getAccumulatorMap())
+          .thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap());
+      TestProcessingTimeService testProcessingTimeService = new TestProcessingTimeService();
+      when(mockTask.getProcessingTimeService()).thenReturn(testProcessingTimeService);
+
+      operator.setup(mockTask, cfg, (Output<StreamRecord<T>>) mock(Output.class));
+    }
+
+    /**
+     * A special {@link RuntimeException} that we throw to signal that the test was successful.
+     */
+    private static class SuccessException extends RuntimeException {
+    }
+  }
+
+  /**
+   * Not parameterized tests.
+   */
+  public static class BasicTest {
+
+    /**
+     * Check serialization a {@link UnboundedSourceWrapper}.
+     */
+    @Test
+    public void testSerialization() throws Exception {
+      final int parallelism = 1;
+      final int numElements = 20;
+      PipelineOptions options = PipelineOptionsFactory.create();
+
+      TestCountingSource source = new TestCountingSource(numElements);
+      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
+          new UnboundedSourceWrapper<>(options, source, parallelism);
+
+      InstantiationUtil.serializeObject(flinkWrapper);
+    }
+
+  }
+
+  private static final class TestingListState<T> implements ListState<T> {
+
+    private final List<T> list = new ArrayList<>();
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public Iterable<T> get() throws Exception {
+      return list;
+    }
+
+    @Override
+    public void add(T value) throws Exception {
+      list.add(value);
+    }
+
+    public List<T> getList() {
+      return list;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
new file mode 100644
index 0000000..08a1e03
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.streaming;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/resources/log4j-test.properties b/runners/flink/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..4c74d85
--- /dev/null
+++ b/runners/flink/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+log4j.rootLogger=OFF, testlogger
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n


[04/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
new file mode 100644
index 0000000..2ed5024
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.StoppableFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Wrapper for executing {@link BoundedSource BoundedSources} as a Flink Source.
+ */
+public class BoundedSourceWrapper<OutputT>
+    extends RichParallelSourceFunction<WindowedValue<OutputT>>
+    implements StoppableFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BoundedSourceWrapper.class);
+
+  /**
+   * Keep the options so that we can initialize the readers.
+   */
+  private final SerializedPipelineOptions serializedOptions;
+
+  /**
+   * The split sources. We split them in the constructor to ensure that all parallel
+   * sources are consistent about the split sources.
+   */
+  private List<? extends BoundedSource<OutputT>> splitSources;
+
+  /**
+   * Make it a field so that we can access it in {@link #close()}.
+   */
+  private transient List<BoundedSource.BoundedReader<OutputT>> readers;
+
+  /**
+   * Initialize here and not in run() to prevent races where we cancel a job before run() is
+   * ever called or run() is called after cancel().
+   */
+  private volatile boolean isRunning = true;
+
+  @SuppressWarnings("unchecked")
+  public BoundedSourceWrapper(
+      PipelineOptions pipelineOptions,
+      BoundedSource<OutputT> source,
+      int parallelism) throws Exception {
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+
+    long desiredBundleSize = source.getEstimatedSizeBytes(pipelineOptions) / parallelism;
+
+    // get the splits early. we assume that the generated splits are stable,
+    // this is necessary so that the mapping of state to source is correct
+    // when restoring
+    splitSources = source.split(desiredBundleSize, pipelineOptions);
+  }
+
+  @Override
+  public void run(SourceContext<WindowedValue<OutputT>> ctx) throws Exception {
+
+    // figure out which split sources we're responsible for
+    int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+    int numSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
+
+    List<BoundedSource<OutputT>> localSources = new ArrayList<>();
+
+    for (int i = 0; i < splitSources.size(); i++) {
+      if (i % numSubtasks == subtaskIndex) {
+        localSources.add(splitSources.get(i));
+      }
+    }
+
+    LOG.info("Bounded Flink Source {}/{} is reading from sources: {}",
+        subtaskIndex,
+        numSubtasks,
+        localSources);
+
+    readers = new ArrayList<>();
+    // initialize readers from scratch
+    for (BoundedSource<OutputT> source : localSources) {
+      readers.add(source.createReader(serializedOptions.getPipelineOptions()));
+    }
+
+   if (readers.size() == 1) {
+      // the easy case, we just read from one reader
+      BoundedSource.BoundedReader<OutputT> reader = readers.get(0);
+
+      boolean dataAvailable = reader.start();
+      if (dataAvailable) {
+        emitElement(ctx, reader);
+      }
+
+      while (isRunning) {
+        dataAvailable = reader.advance();
+
+        if (dataAvailable)  {
+          emitElement(ctx, reader);
+        } else {
+          break;
+        }
+      }
+    } else {
+      // a bit more complicated, we are responsible for several readers
+      // loop through them and sleep if none of them had any data
+
+      int currentReader = 0;
+
+      // start each reader and emit data if immediately available
+      for (BoundedSource.BoundedReader<OutputT> reader : readers) {
+        boolean dataAvailable = reader.start();
+        if (dataAvailable) {
+          emitElement(ctx, reader);
+        }
+      }
+
+      // a flag telling us whether any of the readers had data
+      // if no reader had data, sleep for bit
+      boolean hadData = false;
+      while (isRunning && !readers.isEmpty()) {
+        BoundedSource.BoundedReader<OutputT> reader = readers.get(currentReader);
+        boolean dataAvailable = reader.advance();
+
+        if (dataAvailable) {
+          emitElement(ctx, reader);
+          hadData = true;
+        } else {
+          readers.remove(currentReader);
+          currentReader--;
+          if (readers.isEmpty()) {
+            break;
+          }
+        }
+
+        currentReader = (currentReader + 1) % readers.size();
+        if (currentReader == 0 && !hadData) {
+          Thread.sleep(50);
+        } else if (currentReader == 0) {
+          hadData = false;
+        }
+      }
+
+    }
+
+    // emit final Long.MAX_VALUE watermark, just to be sure
+    ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
+  }
+
+  /**
+   * Emit the current element from the given Reader. The reader is guaranteed to have data.
+   */
+  private void emitElement(
+      SourceContext<WindowedValue<OutputT>> ctx,
+      BoundedSource.BoundedReader<OutputT> reader) {
+    // make sure that reader state update and element emission are atomic
+    // with respect to snapshots
+    synchronized (ctx.getCheckpointLock()) {
+
+      OutputT item = reader.getCurrent();
+      Instant timestamp = reader.getCurrentTimestamp();
+
+      WindowedValue<OutputT> windowedValue =
+          WindowedValue.of(item, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+      ctx.collectWithTimestamp(windowedValue, timestamp.getMillis());
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    if (readers != null) {
+      for (BoundedSource.BoundedReader<OutputT> reader: readers) {
+        reader.close();
+      }
+    }
+  }
+
+  @Override
+  public void cancel() {
+    isRunning = false;
+  }
+
+  @Override
+  public void stop() {
+    this.isRunning = false;
+  }
+
+  /**
+   * Visible so that we can check this in tests. Must not be used for anything else.
+   */
+  @VisibleForTesting
+  public List<? extends BoundedSource<OutputT>> getSplitSources() {
+    return splitSources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
new file mode 100644
index 0000000..910a33f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+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;
+
+/**
+ * An example unbounded Beam source that reads input from a socket.
+ * This is used mainly for testing and debugging.
+ * */
+public class UnboundedSocketSource<CheckpointMarkT extends UnboundedSource.CheckpointMark>
+    extends UnboundedSource<String, CheckpointMarkT> {
+
+  private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
+
+  private static final long serialVersionUID = 1L;
+
+  private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
+
+  private static final int CONNECTION_TIMEOUT_TIME = 0;
+
+  private final String hostname;
+  private final int port;
+  private final char delimiter;
+  private final long maxNumRetries;
+  private final long delayBetweenRetries;
+
+  public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
+    this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
+  }
+
+  public UnboundedSocketSource(String hostname,
+                               int port,
+                               char delimiter,
+                               long maxNumRetries,
+                               long delayBetweenRetries) {
+    this.hostname = hostname;
+    this.port = port;
+    this.delimiter = delimiter;
+    this.maxNumRetries = maxNumRetries;
+    this.delayBetweenRetries = delayBetweenRetries;
+  }
+
+  public String getHostname() {
+    return this.hostname;
+  }
+
+  public int getPort() {
+    return this.port;
+  }
+
+  public char getDelimiter() {
+    return this.delimiter;
+  }
+
+  public long getMaxNumRetries() {
+    return this.maxNumRetries;
+  }
+
+  public long getDelayBetweenRetries() {
+    return this.delayBetweenRetries;
+  }
+
+  @Override
+  public List<? extends UnboundedSource<String, CheckpointMarkT>> split(
+      int desiredNumSplits,
+      PipelineOptions options) throws Exception {
+    return Collections.<UnboundedSource<String, CheckpointMarkT>>singletonList(this);
+  }
+
+  @Override
+  public UnboundedReader<String> createReader(PipelineOptions options,
+                                              @Nullable CheckpointMarkT checkpointMark) {
+    return new UnboundedSocketReader(this);
+  }
+
+  @Nullable
+  @Override
+  public Coder getCheckpointMarkCoder() {
+    // Flink and Dataflow have different checkpointing mechanisms.
+    // In our case we do not need a coder.
+    return null;
+  }
+
+  @Override
+  public void validate() {
+    checkArgument(port > 0 && port < 65536, "port is out of range");
+    checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), "
+        + "or -1 (infinite retries)");
+    checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
+  }
+
+  @Override
+  public Coder getDefaultOutputCoder() {
+    return DEFAULT_SOCKET_CODER;
+  }
+
+  /**
+   * Unbounded socket reader.
+   */
+  public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
+
+    private final UnboundedSocketSource source;
+
+    private Socket socket;
+    private BufferedReader reader;
+
+    private boolean isRunning;
+
+    private String currentRecord;
+
+    public UnboundedSocketReader(UnboundedSocketSource source) {
+      this.source = source;
+    }
+
+    private void openConnection() throws IOException {
+      this.socket = new Socket();
+      this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()),
+          CONNECTION_TIMEOUT_TIME);
+      this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
+      this.isRunning = true;
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      int attempt = 0;
+      while (!isRunning) {
+        try {
+          openConnection();
+          LOG.info("Connected to server socket " + this.source.getHostname() + ':'
+              + this.source.getPort());
+
+          return advance();
+        } catch (IOException e) {
+          LOG.info("Lost connection to server socket " + this.source.getHostname() + ':'
+              + this.source.getPort() + ". Retrying in "
+              + this.source.getDelayBetweenRetries() + " msecs...");
+
+          if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
+            try {
+              Thread.sleep(this.source.getDelayBetweenRetries());
+            } catch (InterruptedException e1) {
+              e1.printStackTrace();
+            }
+          } else {
+            this.isRunning = false;
+            break;
+          }
+        }
+      }
+      LOG.error("Unable to connect to host " + this.source.getHostname()
+          + " : " + this.source.getPort());
+      return false;
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      final StringBuilder buffer = new StringBuilder();
+      int data;
+      while (isRunning && (data = reader.read()) != -1) {
+        // check if the string is complete
+        if (data != this.source.getDelimiter()) {
+          buffer.append((char) data);
+        } else {
+          if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
+            buffer.setLength(buffer.length() - 1);
+          }
+          this.currentRecord = buffer.toString();
+          buffer.setLength(0);
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public byte[] getCurrentRecordId() throws NoSuchElementException {
+      return new byte[0];
+    }
+
+    @Override
+    public String getCurrent() throws NoSuchElementException {
+      return this.currentRecord;
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() throws NoSuchElementException {
+      return Instant.now();
+    }
+
+    @Override
+    public void close() throws IOException {
+      this.reader.close();
+      this.socket.close();
+      this.isRunning = false;
+      LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":"
+          + this.source.getPort() + ".");
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return Instant.now();
+    }
+
+    @Override
+    public CheckpointMark getCheckpointMark() {
+      return null;
+    }
+
+    @Override
+    public UnboundedSource<String, ?> getCurrentSource() {
+      return this.source;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
new file mode 100644
index 0000000..bb9b58a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -0,0 +1,476 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.StoppableFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Wrapper for executing {@link UnboundedSource UnboundedSources} as a Flink Source.
+ */
+public class UnboundedSourceWrapper<
+    OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+    extends RichParallelSourceFunction<WindowedValue<OutputT>>
+    implements ProcessingTimeCallback, StoppableFunction,
+    CheckpointListener, CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceWrapper.class);
+
+  /**
+   * Keep the options so that we can initialize the localReaders.
+   */
+  private final SerializedPipelineOptions serializedOptions;
+
+  /**
+   * For snapshot and restore.
+   */
+  private final KvCoder<
+      ? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> checkpointCoder;
+
+  /**
+   * The split sources. We split them in the constructor to ensure that all parallel
+   * sources are consistent about the split sources.
+   */
+  private final List<? extends UnboundedSource<OutputT, CheckpointMarkT>> splitSources;
+
+  /**
+   * The local split sources. Assigned at runtime when the wrapper is executed in parallel.
+   */
+  private transient List<UnboundedSource<OutputT, CheckpointMarkT>> localSplitSources;
+
+  /**
+   * The local split readers. Assigned at runtime when the wrapper is executed in parallel.
+   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for
+   * emitting watermarks.
+   */
+  private transient List<UnboundedSource.UnboundedReader<OutputT>> localReaders;
+
+  /**
+   * Flag to indicate whether the source is running.
+   * Initialize here and not in run() to prevent races where we cancel a job before run() is
+   * ever called or run() is called after cancel().
+   */
+  private volatile boolean isRunning = true;
+
+  /**
+   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for registering new
+   * triggers.
+   */
+  private transient StreamingRuntimeContext runtimeContext;
+
+  /**
+   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for emitting
+   * watermarks.
+   */
+  private transient SourceContext<WindowedValue<OutputT>> context;
+
+  /**
+   * Pending checkpoints which have not been acknowledged yet.
+   */
+  private transient LinkedHashMap<Long, List<CheckpointMarkT>> pendingCheckpoints;
+  /**
+   * Keep a maximum of 32 checkpoints for {@code CheckpointMark.finalizeCheckpoint()}.
+   */
+  private static final int MAX_NUMBER_PENDING_CHECKPOINTS = 32;
+
+  private transient ListState<KV<? extends
+      UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT>> stateForCheckpoint;
+
+  /**
+   * false if checkpointCoder is null or no restore state by starting first.
+   */
+  private transient boolean isRestored = false;
+
+  @SuppressWarnings("unchecked")
+  public UnboundedSourceWrapper(
+      PipelineOptions pipelineOptions,
+      UnboundedSource<OutputT, CheckpointMarkT> source,
+      int parallelism) throws Exception {
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+
+    if (source.requiresDeduping()) {
+      LOG.warn("Source {} requires deduping but Flink runner doesn't support this yet.", source);
+    }
+
+    Coder<CheckpointMarkT> checkpointMarkCoder = source.getCheckpointMarkCoder();
+    if (checkpointMarkCoder == null) {
+      LOG.info("No CheckpointMarkCoder specified for this source. Won't create snapshots.");
+      checkpointCoder = null;
+    } else {
+
+      Coder<? extends UnboundedSource<OutputT, CheckpointMarkT>> sourceCoder =
+          (Coder) SerializableCoder.of(new TypeDescriptor<UnboundedSource>() {
+          });
+
+      checkpointCoder = KvCoder.of(sourceCoder, checkpointMarkCoder);
+    }
+
+    // get the splits early. we assume that the generated splits are stable,
+    // this is necessary so that the mapping of state to source is correct
+    // when restoring
+    splitSources = source.split(parallelism, pipelineOptions);
+  }
+
+
+  /**
+   * Initialize and restore state before starting execution of the source.
+   */
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
+
+    // figure out which split sources we're responsible for
+    int subtaskIndex = runtimeContext.getIndexOfThisSubtask();
+    int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
+
+    localSplitSources = new ArrayList<>();
+    localReaders = new ArrayList<>();
+
+    pendingCheckpoints = new LinkedHashMap<>();
+
+    if (isRestored) {
+      // restore the splitSources from the checkpoint to ensure consistent ordering
+      for (KV<? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> restored:
+          stateForCheckpoint.get()) {
+        localSplitSources.add(restored.getKey());
+        localReaders.add(restored.getKey().createReader(
+            serializedOptions.getPipelineOptions(), restored.getValue()));
+      }
+    } else {
+      // initialize localReaders and localSources from scratch
+      for (int i = 0; i < splitSources.size(); i++) {
+        if (i % numSubtasks == subtaskIndex) {
+          UnboundedSource<OutputT, CheckpointMarkT> source =
+              splitSources.get(i);
+          UnboundedSource.UnboundedReader<OutputT> reader =
+              source.createReader(serializedOptions.getPipelineOptions(), null);
+          localSplitSources.add(source);
+          localReaders.add(reader);
+        }
+      }
+    }
+
+    LOG.info("Unbounded Flink Source {}/{} is reading from sources: {}",
+        subtaskIndex,
+        numSubtasks,
+        localSplitSources);
+  }
+
+  @Override
+  public void run(SourceContext<WindowedValue<OutputT>> ctx) throws Exception {
+
+    context = ctx;
+
+    if (localReaders.size() == 0) {
+      // do nothing, but still look busy ...
+      // also, output a Long.MAX_VALUE watermark since we know that we're not
+      // going to emit anything
+      // we can't return here since Flink requires that all operators stay up,
+      // otherwise checkpointing would not work correctly anymore
+      ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
+
+      // wait until this is canceled
+      final Object waitLock = new Object();
+      while (isRunning) {
+        try {
+          // Flink will interrupt us at some point
+          //noinspection SynchronizationOnLocalVariableOrMethodParameter
+          synchronized (waitLock) {
+            // don't wait indefinitely, in case something goes horribly wrong
+            waitLock.wait(1000);
+          }
+        } catch (InterruptedException e) {
+          if (!isRunning) {
+            // restore the interrupted state, and fall through the loop
+            Thread.currentThread().interrupt();
+          }
+        }
+      }
+    } else if (localReaders.size() == 1) {
+      // the easy case, we just read from one reader
+      UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(0);
+
+      boolean dataAvailable = reader.start();
+      if (dataAvailable) {
+        emitElement(ctx, reader);
+      }
+
+      setNextWatermarkTimer(this.runtimeContext);
+
+      while (isRunning) {
+        dataAvailable = reader.advance();
+
+        if (dataAvailable)  {
+          emitElement(ctx, reader);
+        } else {
+          Thread.sleep(50);
+        }
+      }
+    } else {
+      // a bit more complicated, we are responsible for several localReaders
+      // loop through them and sleep if none of them had any data
+
+      int numReaders = localReaders.size();
+      int currentReader = 0;
+
+      // start each reader and emit data if immediately available
+      for (UnboundedSource.UnboundedReader<OutputT> reader : localReaders) {
+        boolean dataAvailable = reader.start();
+        if (dataAvailable) {
+          emitElement(ctx, reader);
+        }
+      }
+
+      // a flag telling us whether any of the localReaders had data
+      // if no reader had data, sleep for bit
+      boolean hadData = false;
+      while (isRunning) {
+        UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(currentReader);
+        boolean dataAvailable = reader.advance();
+
+        if (dataAvailable) {
+          emitElement(ctx, reader);
+          hadData = true;
+        }
+
+        currentReader = (currentReader + 1) % numReaders;
+        if (currentReader == 0 && !hadData) {
+          Thread.sleep(50);
+        } else if (currentReader == 0) {
+          hadData = false;
+        }
+      }
+
+    }
+  }
+
+  /**
+   * Emit the current element from the given Reader. The reader is guaranteed to have data.
+   */
+  private void emitElement(
+      SourceContext<WindowedValue<OutputT>> ctx,
+      UnboundedSource.UnboundedReader<OutputT> reader) {
+    // make sure that reader state update and element emission are atomic
+    // with respect to snapshots
+    synchronized (ctx.getCheckpointLock()) {
+
+      OutputT item = reader.getCurrent();
+      Instant timestamp = reader.getCurrentTimestamp();
+
+      WindowedValue<OutputT> windowedValue =
+          WindowedValue.of(item, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+      ctx.collectWithTimestamp(windowedValue, timestamp.getMillis());
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    if (localReaders != null) {
+      for (UnboundedSource.UnboundedReader<OutputT> reader: localReaders) {
+        reader.close();
+      }
+    }
+  }
+
+  @Override
+  public void cancel() {
+    isRunning = false;
+  }
+
+  @Override
+  public void stop() {
+    isRunning = false;
+  }
+
+  // ------------------------------------------------------------------------
+  //  Checkpoint and restore
+  // ------------------------------------------------------------------------
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
+    if (!isRunning) {
+      LOG.debug("snapshotState() called on closed source");
+    } else {
+
+      if (checkpointCoder == null) {
+        // no checkpoint coder available in this source
+        return;
+      }
+
+      stateForCheckpoint.clear();
+
+      long checkpointId = functionSnapshotContext.getCheckpointId();
+
+      // we checkpoint the sources along with the CheckpointMarkT to ensure
+      // than we have a correct mapping of checkpoints to sources when
+      // restoring
+      List<CheckpointMarkT> checkpointMarks = new ArrayList<>(localSplitSources.size());
+
+      for (int i = 0; i < localSplitSources.size(); i++) {
+        UnboundedSource<OutputT, CheckpointMarkT> source = localSplitSources.get(i);
+        UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(i);
+
+        @SuppressWarnings("unchecked")
+        CheckpointMarkT mark = (CheckpointMarkT) reader.getCheckpointMark();
+        checkpointMarks.add(mark);
+        KV<UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> kv =
+            KV.of(source, mark);
+        stateForCheckpoint.add(kv);
+      }
+
+      // cleanup old pending checkpoints and add new checkpoint
+      int diff = pendingCheckpoints.size() - MAX_NUMBER_PENDING_CHECKPOINTS;
+      if (diff >= 0) {
+        for (Iterator<Long> iterator = pendingCheckpoints.keySet().iterator();
+             diff >= 0;
+             diff--) {
+          iterator.next();
+          iterator.remove();
+        }
+      }
+      pendingCheckpoints.put(checkpointId, checkpointMarks);
+
+    }
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    if (checkpointCoder == null) {
+      // no checkpoint coder available in this source
+      return;
+    }
+
+    OperatorStateStore stateStore = context.getOperatorStateStore();
+    CoderTypeInformation<
+        KV<? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT>>
+        typeInformation = (CoderTypeInformation) new CoderTypeInformation<>(checkpointCoder);
+    stateForCheckpoint = stateStore.getOperatorState(
+        new ListStateDescriptor<>(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME,
+            typeInformation.createSerializer(new ExecutionConfig())));
+
+    if (context.isRestored()) {
+      isRestored = true;
+      LOG.info("Having restore state in the UnbounedSourceWrapper.");
+    } else {
+      LOG.info("No restore state for UnbounedSourceWrapper.");
+    }
+  }
+
+  @Override
+  public void onProcessingTime(long timestamp) throws Exception {
+    if (this.isRunning) {
+      synchronized (context.getCheckpointLock()) {
+        // find minimum watermark over all localReaders
+        long watermarkMillis = Long.MAX_VALUE;
+        for (UnboundedSource.UnboundedReader<OutputT> reader: localReaders) {
+          Instant watermark = reader.getWatermark();
+          if (watermark != null) {
+            watermarkMillis = Math.min(watermark.getMillis(), watermarkMillis);
+          }
+        }
+        context.emitWatermark(new Watermark(watermarkMillis));
+      }
+      setNextWatermarkTimer(this.runtimeContext);
+    }
+  }
+
+  private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
+    if (this.isRunning) {
+      long watermarkInterval =  runtime.getExecutionConfig().getAutoWatermarkInterval();
+      long timeToNextWatermark = getTimeToNextWatermark(watermarkInterval);
+      runtime.getProcessingTimeService().registerTimer(timeToNextWatermark, this);
+    }
+  }
+
+  private long getTimeToNextWatermark(long watermarkInterval) {
+    return System.currentTimeMillis() + watermarkInterval;
+  }
+
+  /**
+   * Visible so that we can check this in tests. Must not be used for anything else.
+   */
+  @VisibleForTesting
+  public List<? extends UnboundedSource<OutputT, CheckpointMarkT>> getSplitSources() {
+    return splitSources;
+  }
+
+  /**
+   * Visible so that we can check this in tests. Must not be used for anything else.
+   */
+  @VisibleForTesting
+  public List<? extends UnboundedSource<OutputT, CheckpointMarkT>> getLocalSplitSources() {
+    return localSplitSources;
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+
+    List<CheckpointMarkT> checkpointMarks = pendingCheckpoints.get(checkpointId);
+
+    if (checkpointMarks != null) {
+
+      // remove old checkpoints including the current one
+      Iterator<Long> iterator = pendingCheckpoints.keySet().iterator();
+      long currentId;
+      do {
+        currentId = iterator.next();
+        iterator.remove();
+      } while (currentId != checkpointId);
+
+      // confirm all marks
+      for (CheckpointMarkT mark : checkpointMarks) {
+        mark.finalizeCheckpoint();
+      }
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
new file mode 100644
index 0000000..b431ce7
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.io;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
new file mode 100644
index 0000000..0674871
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
new file mode 100644
index 0000000..3203446
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
@@ -0,0 +1,865 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.CombineContextFactory;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MapState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.SetState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+
+/**
+ * {@link StateInternals} that uses a Flink {@link DefaultOperatorStateBackend}
+ * to manage the broadcast state.
+ * The state is the same on all parallel instances of the operator.
+ * So we just need store state of operator-0 in OperatorStateBackend.
+ *
+ * <p>Note: Ignore index of key.
+ * Mainly for SideInputs.
+ */
+public class FlinkBroadcastStateInternals<K> implements StateInternals<K> {
+
+  private int indexInSubtaskGroup;
+  private final DefaultOperatorStateBackend stateBackend;
+  // stateName -> <namespace, state>
+  private Map<String, Map<String, ?>> stateForNonZeroOperator;
+
+  public FlinkBroadcastStateInternals(int indexInSubtaskGroup, OperatorStateBackend stateBackend) {
+    //TODO flink do not yet expose through public API
+    this.stateBackend = (DefaultOperatorStateBackend) stateBackend;
+    this.indexInSubtaskGroup = indexInSubtaskGroup;
+    if (indexInSubtaskGroup != 0) {
+      stateForNonZeroOperator = new HashMap<>();
+    }
+  }
+
+  @Override
+  public K getKey() {
+    return null;
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address) {
+
+    return state(namespace, address, StateContexts.nullContext());
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address,
+      final StateContext<?> context) {
+
+    return address.bind(new StateTag.StateBinder<K>() {
+
+      @Override
+      public <T> ValueState<T> bindValue(
+          StateTag<? super K, ValueState<T>> address,
+          Coder<T> coder) {
+
+        return new FlinkBroadcastValueState<>(stateBackend, address, namespace, coder);
+      }
+
+      @Override
+      public <T> BagState<T> bindBag(
+          StateTag<? super K, BagState<T>> address,
+          Coder<T> elemCoder) {
+
+        return new FlinkBroadcastBagState<>(stateBackend, address, namespace, elemCoder);
+      }
+
+      @Override
+      public <T> SetState<T> bindSet(
+          StateTag<? super K, SetState<T>> address,
+          Coder<T> elemCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", SetState.class.getSimpleName()));
+      }
+
+      @Override
+      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
+          StateTag<? super K, MapState<KeyT, ValueT>> spec,
+          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", MapState.class.getSimpleName()));
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT>
+      bindCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+
+        return new FlinkCombiningState<>(
+            stateBackend, address, combineFn, namespace, accumCoder);
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+        return new FlinkKeyedCombiningState<>(
+            stateBackend,
+            address,
+            combineFn,
+            namespace,
+            accumCoder,
+            FlinkBroadcastStateInternals.this);
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          CombineWithContext.KeyedCombineFnWithContext<
+              ? super K, InputT, AccumT, OutputT> combineFn) {
+        return new FlinkCombiningStateWithContext<>(
+            stateBackend,
+            address,
+            combineFn,
+            namespace,
+            accumCoder,
+            FlinkBroadcastStateInternals.this,
+            CombineContextFactory.createFromStateContext(context));
+      }
+
+      @Override
+      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
+          StateTag<? super K, WatermarkHoldState<W>> address,
+          OutputTimeFn<? super W> outputTimeFn) {
+         throw new UnsupportedOperationException(
+             String.format("%s is not supported", WatermarkHoldState.class.getSimpleName()));
+      }
+    });
+  }
+
+  /**
+   * 1. The way we would use it is to only checkpoint anything from the operator
+   * with subtask index 0 because we assume that the state is the same on all
+   * parallel instances of the operator.
+   *
+   * <p>2. Use map to support namespace.
+   */
+  private abstract class AbstractBroadcastState<T> {
+
+    private String name;
+    private final StateNamespace namespace;
+    private final ListStateDescriptor<Map<String, T>> flinkStateDescriptor;
+    private final DefaultOperatorStateBackend flinkStateBackend;
+
+    AbstractBroadcastState(
+        DefaultOperatorStateBackend flinkStateBackend,
+        String name,
+        StateNamespace namespace,
+        Coder<T> coder) {
+      this.name = name;
+
+      this.namespace = namespace;
+      this.flinkStateBackend = flinkStateBackend;
+
+      CoderTypeInformation<Map<String, T>> typeInfo =
+          new CoderTypeInformation<>(MapCoder.of(StringUtf8Coder.of(), coder));
+
+      flinkStateDescriptor = new ListStateDescriptor<>(name,
+          typeInfo.createSerializer(new ExecutionConfig()));
+    }
+
+    /**
+     * Get map(namespce->T) from index 0.
+     */
+    Map<String, T> getMap() throws Exception {
+      if (indexInSubtaskGroup == 0) {
+        return getMapFromBroadcastState();
+      } else {
+        Map<String, T> result = (Map<String, T>) stateForNonZeroOperator.get(name);
+        // maybe restore from BroadcastState of Operator-0
+        if (result == null) {
+          result = getMapFromBroadcastState();
+          if (result != null) {
+            stateForNonZeroOperator.put(name, result);
+            // we don't need it anymore, must clear it.
+            flinkStateBackend.getBroadcastOperatorState(
+                flinkStateDescriptor).clear();
+          }
+        }
+        return result;
+      }
+    }
+
+    Map<String, T> getMapFromBroadcastState() throws Exception {
+      ListState<Map<String, T>> state = flinkStateBackend.getBroadcastOperatorState(
+          flinkStateDescriptor);
+      Iterable<Map<String, T>> iterable = state.get();
+      Map<String, T> ret = null;
+      if (iterable != null) {
+        // just use index 0
+        Iterator<Map<String, T>> iterator = iterable.iterator();
+        if (iterator.hasNext()) {
+          ret = iterator.next();
+        }
+      }
+      return ret;
+    }
+
+    /**
+     * Update map(namespce->T) from index 0.
+     */
+    void updateMap(Map<String, T> map) throws Exception {
+      if (indexInSubtaskGroup == 0) {
+        ListState<Map<String, T>> state = flinkStateBackend.getBroadcastOperatorState(
+            flinkStateDescriptor);
+        state.clear();
+        if (map.size() > 0) {
+          state.add(map);
+        }
+      } else {
+        if (map.size() == 0) {
+          stateForNonZeroOperator.remove(name);
+          // updateMap is always behind getMap,
+          // getMap will clear map in BroadcastOperatorState,
+          // we don't need clear here.
+        } else {
+          stateForNonZeroOperator.put(name, map);
+        }
+      }
+    }
+
+    void writeInternal(T input) {
+      try {
+        Map<String, T> map = getMap();
+        if (map == null) {
+          map = new HashMap<>();
+        }
+        map.put(namespace.stringKey(), input);
+        updateMap(map);
+      } catch (Exception e) {
+        throw new RuntimeException("Error updating state.", e);
+      }
+    }
+
+    T readInternal() {
+      try {
+        Map<String, T> map = getMap();
+        if (map == null) {
+          return null;
+        } else {
+          return map.get(namespace.stringKey());
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    void clearInternal() {
+      try {
+        Map<String, T> map = getMap();
+        if (map != null) {
+          map.remove(namespace.stringKey());
+          updateMap(map);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+  }
+
+  private class FlinkBroadcastValueState<K, T>
+      extends AbstractBroadcastState<T> implements ValueState<T> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, ValueState<T>> address;
+
+    FlinkBroadcastValueState(
+        DefaultOperatorStateBackend flinkStateBackend,
+        StateTag<? super K, ValueState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+      super(flinkStateBackend, address.getId(), namespace, coder);
+
+      this.namespace = namespace;
+      this.address = address;
+
+    }
+
+    @Override
+    public void write(T input) {
+      writeInternal(input);
+    }
+
+    @Override
+    public ValueState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public T read() {
+      return readInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkBroadcastValueState<?, ?> that = (FlinkBroadcastValueState<?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+  }
+
+  private class FlinkBroadcastBagState<K, T> extends AbstractBroadcastState<List<T>>
+      implements BagState<T> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, BagState<T>> address;
+
+    FlinkBroadcastBagState(
+        DefaultOperatorStateBackend flinkStateBackend,
+        StateTag<? super K, BagState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+      super(flinkStateBackend, address.getId(), namespace, ListCoder.of(coder));
+
+      this.namespace = namespace;
+      this.address = address;
+    }
+
+    @Override
+    public void add(T input) {
+      List<T> list = readInternal();
+      if (list == null) {
+        list = new ArrayList<>();
+      }
+      list.add(input);
+      writeInternal(list);
+    }
+
+    @Override
+    public BagState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public Iterable<T> read() {
+      List<T> result = readInternal();
+      return result != null ? result : Collections.<T>emptyList();
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            List<T> result = readInternal();
+            return result == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkBroadcastBagState<?, ?> that = (FlinkBroadcastBagState<?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private class FlinkCombiningState<K, InputT, AccumT, OutputT>
+      extends AbstractBroadcastState<AccumT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
+
+    FlinkCombiningState(
+        DefaultOperatorStateBackend flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder) {
+      super(flinkStateBackend, address.getId(), namespace, accumCoder);
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      AccumT current = readInternal();
+      if (current == null) {
+        current = combineFn.createAccumulator();
+      }
+      current = combineFn.addInput(current, value);
+      writeInternal(current);
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      AccumT current = readInternal();
+
+      if (current == null) {
+        writeInternal(accum);
+      } else {
+        current = combineFn.mergeAccumulators(Arrays.asList(current, accum));
+        writeInternal(current);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      return readInternal();
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(accumulators);
+    }
+
+    @Override
+    public OutputT read() {
+      AccumT accum = readInternal();
+      if (accum != null) {
+        return combineFn.extractOutput(accum);
+      } else {
+        return combineFn.extractOutput(combineFn.createAccumulator());
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return readInternal() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkCombiningState<?, ?, ?, ?> that =
+          (FlinkCombiningState<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private class FlinkKeyedCombiningState<K, InputT, AccumT, OutputT>
+      extends AbstractBroadcastState<AccumT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn;
+    private final FlinkBroadcastStateInternals<K> flinkStateInternals;
+
+    FlinkKeyedCombiningState(
+        DefaultOperatorStateBackend flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder,
+        FlinkBroadcastStateInternals<K> flinkStateInternals) {
+      super(flinkStateBackend, address.getId(), namespace, accumCoder);
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+      this.flinkStateInternals = flinkStateInternals;
+
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      try {
+        AccumT current = readInternal();
+        if (current == null) {
+          current = combineFn.createAccumulator(flinkStateInternals.getKey());
+        }
+        current = combineFn.addInput(flinkStateInternals.getKey(), current, value);
+        writeInternal(current);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state." , e);
+      }
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      try {
+        AccumT current = readInternal();
+        if (current == null) {
+          writeInternal(accum);
+        } else {
+          current = combineFn.mergeAccumulators(
+              flinkStateInternals.getKey(),
+              Arrays.asList(current, accum));
+          writeInternal(current);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state.", e);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      try {
+        return readInternal();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators);
+    }
+
+    @Override
+    public OutputT read() {
+      try {
+        AccumT accum = readInternal();
+        if (accum != null) {
+          return combineFn.extractOutput(flinkStateInternals.getKey(), accum);
+        } else {
+          return combineFn.extractOutput(
+              flinkStateInternals.getKey(),
+              combineFn.createAccumulator(flinkStateInternals.getKey()));
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return readInternal() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkKeyedCombiningState<?, ?, ?, ?> that =
+          (FlinkKeyedCombiningState<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private class FlinkCombiningStateWithContext<K, InputT, AccumT, OutputT>
+      extends AbstractBroadcastState<AccumT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final CombineWithContext.KeyedCombineFnWithContext<
+        ? super K, InputT, AccumT, OutputT> combineFn;
+    private final FlinkBroadcastStateInternals<K> flinkStateInternals;
+    private final CombineWithContext.Context context;
+
+    FlinkCombiningStateWithContext(
+        DefaultOperatorStateBackend flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        CombineWithContext.KeyedCombineFnWithContext<
+            ? super K, InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder,
+        FlinkBroadcastStateInternals<K> flinkStateInternals,
+        CombineWithContext.Context context) {
+      super(flinkStateBackend, address.getId(), namespace, accumCoder);
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+      this.flinkStateInternals = flinkStateInternals;
+      this.context = context;
+
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      try {
+        AccumT current = readInternal();
+        if (current == null) {
+          current = combineFn.createAccumulator(flinkStateInternals.getKey(), context);
+        }
+        current = combineFn.addInput(flinkStateInternals.getKey(), current, value, context);
+        writeInternal(current);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state." , e);
+      }
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      try {
+
+        AccumT current = readInternal();
+        if (current == null) {
+          writeInternal(accum);
+        } else {
+          current = combineFn.mergeAccumulators(
+              flinkStateInternals.getKey(),
+              Arrays.asList(current, accum),
+              context);
+          writeInternal(current);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state.", e);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      try {
+        return readInternal();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators, context);
+    }
+
+    @Override
+    public OutputT read() {
+      try {
+        AccumT accum = readInternal();
+        return combineFn.extractOutput(flinkStateInternals.getKey(), accum, context);
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return readInternal() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkCombiningStateWithContext<?, ?, ?, ?> that =
+          (FlinkCombiningStateWithContext<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
new file mode 100644
index 0000000..24b340e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateTag;
+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.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.CoderUtils;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MapState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.SetState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.streaming.api.operators.HeapInternalTimerService;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * {@link StateInternals} that uses {@link KeyGroupCheckpointedOperator}
+ * to checkpoint state.
+ *
+ * <p>Note:
+ * Ignore index of key.
+ * Just implement BagState.
+ *
+ * <p>Reference from {@link HeapInternalTimerService} to the local key-group range.
+ */
+public class FlinkKeyGroupStateInternals<K> implements StateInternals<K> {
+
+  private final Coder<K> keyCoder;
+  private final KeyGroupsList localKeyGroupRange;
+  private KeyedStateBackend keyedStateBackend;
+  private final int localKeyGroupRangeStartIdx;
+
+  // stateName -> namespace -> (valueCoder, value)
+  private final Map<String, Tuple2<Coder<?>, Map<String, ?>>>[] stateTables;
+
+  public FlinkKeyGroupStateInternals(
+      Coder<K> keyCoder,
+      KeyedStateBackend keyedStateBackend) {
+    this.keyCoder = keyCoder;
+    this.keyedStateBackend = keyedStateBackend;
+    this.localKeyGroupRange = keyedStateBackend.getKeyGroupRange();
+    // find the starting index of the local key-group range
+    int startIdx = Integer.MAX_VALUE;
+    for (Integer keyGroupIdx : localKeyGroupRange) {
+      startIdx = Math.min(keyGroupIdx, startIdx);
+    }
+    this.localKeyGroupRangeStartIdx = startIdx;
+    stateTables = (Map<String, Tuple2<Coder<?>, Map<String, ?>>>[])
+        new Map[localKeyGroupRange.getNumberOfKeyGroups()];
+    for (int i = 0; i < stateTables.length; i++) {
+      stateTables[i] = new HashMap<>();
+    }
+  }
+
+  @Override
+  public K getKey() {
+    ByteBuffer keyBytes = (ByteBuffer) keyedStateBackend.getCurrentKey();
+    try {
+      return CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array());
+    } catch (CoderException e) {
+      throw new RuntimeException("Error decoding key.", e);
+    }
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address) {
+
+    return state(namespace, address, StateContexts.nullContext());
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address,
+      final StateContext<?> context) {
+
+    return address.bind(new StateTag.StateBinder<K>() {
+
+      @Override
+      public <T> ValueState<T> bindValue(
+          StateTag<? super K, ValueState<T>> address,
+          Coder<T> coder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", ValueState.class.getSimpleName()));
+      }
+
+      @Override
+      public <T> BagState<T> bindBag(
+          StateTag<? super K, BagState<T>> address,
+          Coder<T> elemCoder) {
+
+        return new FlinkKeyGroupBagState<>(address, namespace, elemCoder);
+      }
+
+      @Override
+      public <T> SetState<T> bindSet(
+          StateTag<? super K, SetState<T>> address,
+          Coder<T> elemCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", SetState.class.getSimpleName()));
+      }
+
+      @Override
+      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
+          StateTag<? super K, MapState<KeyT, ValueT>> spec,
+          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", MapState.class.getSimpleName()));
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT>
+      bindCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException("bindCombiningValue is not supported.");
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported.");
+
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          CombineWithContext.KeyedCombineFnWithContext<
+              ? super K, InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException(
+            "bindKeyedCombiningValueWithContext is not supported.");
+      }
+
+      @Override
+      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
+          StateTag<? super K, WatermarkHoldState<W>> address,
+          OutputTimeFn<? super W> outputTimeFn) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", CombiningState.class.getSimpleName()));
+      }
+    });
+  }
+
+  /**
+   * Reference from {@link Combine.CombineFn}.
+   *
+   * <p>Accumulators are stored in each KeyGroup, call addInput() when a element comes,
+   * call extractOutput() to produce the desired value when need to read data.
+   */
+  interface KeyGroupCombiner<InputT, AccumT, OutputT> {
+
+    /**
+     * Returns a new, mutable accumulator value, representing the accumulation
+     * of zero input values.
+     */
+    AccumT createAccumulator();
+
+    /**
+     * Adds the given input value to the given accumulator, returning the
+     * new accumulator value.
+     */
+    AccumT addInput(AccumT accumulator, InputT input);
+
+    /**
+     * Returns the output value that is the result of all accumulators from KeyGroups
+     * that are assigned to this operator.
+     */
+    OutputT extractOutput(Iterable<AccumT> accumulators);
+  }
+
+  private abstract class AbstractKeyGroupState<InputT, AccumT, OutputT> {
+
+    private String stateName;
+    private String namespace;
+    private Coder<AccumT> coder;
+    private KeyGroupCombiner<InputT, AccumT, OutputT> keyGroupCombiner;
+
+    AbstractKeyGroupState(
+        String stateName,
+        String namespace,
+        Coder<AccumT> coder,
+        KeyGroupCombiner<InputT, AccumT, OutputT> keyGroupCombiner) {
+      this.stateName = stateName;
+      this.namespace = namespace;
+      this.coder = coder;
+      this.keyGroupCombiner = keyGroupCombiner;
+    }
+
+    /**
+     * Choose keyGroup of input and addInput to accumulator.
+     */
+    void addInput(InputT input) {
+      int keyGroupIdx = keyedStateBackend.getCurrentKeyGroupIndex();
+      int localIdx = getIndexForKeyGroup(keyGroupIdx);
+      Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
+      Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
+      if (tuple2 == null) {
+        tuple2 = new Tuple2<>();
+        tuple2.f0 = coder;
+        tuple2.f1 = new HashMap<>();
+        stateTable.put(stateName, tuple2);
+      }
+      Map<String, AccumT> map = (Map<String, AccumT>) tuple2.f1;
+      AccumT accumulator = map.get(namespace);
+      if (accumulator == null) {
+        accumulator = keyGroupCombiner.createAccumulator();
+      }
+      accumulator = keyGroupCombiner.addInput(accumulator, input);
+      map.put(namespace, accumulator);
+    }
+
+    /**
+     * Get all accumulators and invoke extractOutput().
+     */
+    OutputT extractOutput() {
+      List<AccumT> accumulators = new ArrayList<>(stateTables.length);
+      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
+        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
+        if (tuple2 != null) {
+          AccumT accumulator = (AccumT) tuple2.f1.get(namespace);
+          if (accumulator != null) {
+            accumulators.add(accumulator);
+          }
+        }
+      }
+      return keyGroupCombiner.extractOutput(accumulators);
+    }
+
+    /**
+     * Find the first accumulator and return immediately.
+     */
+    boolean isEmptyInternal() {
+      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
+        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
+        if (tuple2 != null) {
+          AccumT accumulator = (AccumT) tuple2.f1.get(namespace);
+          if (accumulator != null) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+
+    /**
+     * Clear accumulators and clean empty map.
+     */
+    void clearInternal() {
+      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
+        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
+        if (tuple2 != null) {
+          tuple2.f1.remove(namespace);
+          if (tuple2.f1.size() == 0) {
+            stateTable.remove(stateName);
+          }
+        }
+      }
+    }
+
+  }
+
+  private int getIndexForKeyGroup(int keyGroupIdx) {
+    checkArgument(localKeyGroupRange.contains(keyGroupIdx),
+        "Key Group " + keyGroupIdx + " does not belong to the local range.");
+    return keyGroupIdx - this.localKeyGroupRangeStartIdx;
+  }
+
+  private class KeyGroupBagCombiner<T> implements KeyGroupCombiner<T, List<T>, Iterable<T>> {
+
+    @Override
+    public List<T> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<T> addInput(List<T> accumulator, T input) {
+      accumulator.add(input);
+      return accumulator;
+    }
+
+    @Override
+    public Iterable<T> extractOutput(Iterable<List<T>> accumulators) {
+      List<T> result = new ArrayList<>();
+      // maybe can return an unmodifiable view.
+      for (List<T> list : accumulators) {
+        result.addAll(list);
+      }
+      return result;
+    }
+  }
+
+  private class FlinkKeyGroupBagState<T> extends AbstractKeyGroupState<T, List<T>, Iterable<T>>
+      implements BagState<T> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, BagState<T>> address;
+
+    FlinkKeyGroupBagState(
+        StateTag<? super K, BagState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+      super(address.getId(), namespace.stringKey(), ListCoder.of(coder),
+          new KeyGroupBagCombiner<T>());
+      this.namespace = namespace;
+      this.address = address;
+    }
+
+    @Override
+    public void add(T input) {
+      addInput(input);
+    }
+
+    @Override
+    public BagState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public Iterable<T> read() {
+      Iterable<T> result = extractOutput();
+      return result != null ? result : Collections.<T>emptyList();
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return isEmptyInternal();
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      clearInternal();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkKeyGroupBagState<?> that = (FlinkKeyGroupBagState<?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  /**
+   * Snapshots the state {@code (stateName -> (valueCoder && (namespace -> value)))} for a given
+   * {@code keyGroupIdx}.
+   *
+   * @param keyGroupIdx the id of the key-group to be put in the snapshot.
+   * @param out the stream to write to.
+   */
+  public void snapshotKeyGroupState(int keyGroupIdx, DataOutputStream out) throws Exception {
+    int localIdx = getIndexForKeyGroup(keyGroupIdx);
+    Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
+    Preconditions.checkState(stateTable.size() <= Short.MAX_VALUE,
+        "Too many States: " + stateTable.size() + ". Currently at most "
+            + Short.MAX_VALUE + " states are supported");
+    out.writeShort(stateTable.size());
+    for (Map.Entry<String, Tuple2<Coder<?>, Map<String, ?>>> entry : stateTable.entrySet()) {
+      out.writeUTF(entry.getKey());
+      Coder coder = entry.getValue().f0;
+      InstantiationUtil.serializeObject(out, coder);
+      Map<String, ?> map = entry.getValue().f1;
+      out.writeInt(map.size());
+      for (Map.Entry<String, ?> entry1 : map.entrySet()) {
+        StringUtf8Coder.of().encode(entry1.getKey(), out, Context.NESTED);
+        coder.encode(entry1.getValue(), out, Context.NESTED);
+      }
+    }
+  }
+
+  /**
+   * Restore the state {@code (stateName -> (valueCoder && (namespace -> value)))}
+   * for a given {@code keyGroupIdx}.
+   *
+   * @param keyGroupIdx the id of the key-group to be put in the snapshot.
+   * @param in the stream to read from.
+   * @param userCodeClassLoader the class loader that will be used to deserialize
+   *                            the valueCoder.
+   */
+  public void restoreKeyGroupState(int keyGroupIdx, DataInputStream in,
+                                   ClassLoader userCodeClassLoader) throws Exception {
+    int localIdx = getIndexForKeyGroup(keyGroupIdx);
+    Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
+    int numStates = in.readShort();
+    for (int i = 0; i < numStates; ++i) {
+      String stateName = in.readUTF();
+      Coder coder = InstantiationUtil.deserializeObject(in, userCodeClassLoader);
+      Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
+      if (tuple2 == null) {
+        tuple2 = new Tuple2<>();
+        tuple2.f0 = coder;
+        tuple2.f1 = new HashMap<>();
+        stateTable.put(stateName, tuple2);
+      }
+      Map<String, Object> map = (Map<String, Object>) tuple2.f1;
+      int mapSize = in.readInt();
+      for (int j = 0; j < mapSize; j++) {
+        String namespace = StringUtf8Coder.of().decode(in, Context.NESTED);
+        Object value = coder.decode(in, Context.NESTED);
+        map.put(namespace, value);
+      }
+    }
+  }
+
+}


[08/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
new file mode 100644
index 0000000..ba00036
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.CollectionEnvironment;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The class that instantiates and manages the execution of a given job.
+ * Depending on if the job is a Streaming or Batch processing one, it creates
+ * the adequate execution environment ({@link ExecutionEnvironment}
+ * or {@link StreamExecutionEnvironment}), the necessary {@link FlinkPipelineTranslator}
+ * ({@link FlinkBatchPipelineTranslator} or {@link FlinkStreamingPipelineTranslator}) to
+ * transform the Beam job into a Flink one, and executes the (translated) job.
+ */
+class FlinkPipelineExecutionEnvironment {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
+
+  private final FlinkPipelineOptions options;
+
+  /**
+   * The Flink Batch execution environment. This is instantiated to either a
+   * {@link org.apache.flink.api.java.CollectionEnvironment},
+   * a {@link org.apache.flink.api.java.LocalEnvironment} or
+   * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
+   * options.
+   */
+  private ExecutionEnvironment flinkBatchEnv;
+
+  /**
+   * The Flink Streaming execution environment. This is instantiated to either a
+   * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
+   * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
+   * on the configuration options, and more specifically, the url of the master.
+   */
+  private StreamExecutionEnvironment flinkStreamEnv;
+
+  /**
+   * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
+   * provided {@link FlinkPipelineOptions}.
+   *
+   * @param options the user-defined pipeline options.
+   * */
+  FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
+    this.options = checkNotNull(options);
+  }
+
+  /**
+   * 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 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.
+   * */
+  public void translate(FlinkRunner flinkRunner, Pipeline pipeline) {
+    this.flinkBatchEnv = null;
+    this.flinkStreamEnv = null;
+
+    PipelineTranslationOptimizer optimizer =
+        new PipelineTranslationOptimizer(TranslationMode.BATCH, options);
+
+    optimizer.translate(pipeline);
+    TranslationMode translationMode = optimizer.getTranslationMode();
+
+    FlinkPipelineTranslator translator;
+    if (translationMode == TranslationMode.STREAMING) {
+      this.flinkStreamEnv = createStreamExecutionEnvironment();
+      translator = new FlinkStreamingPipelineTranslator(flinkRunner, flinkStreamEnv, options);
+    } else {
+      this.flinkBatchEnv = createBatchExecutionEnvironment();
+      translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
+    }
+
+    translator.translate(pipeline);
+  }
+
+  /**
+   * Launches the program execution.
+   * */
+  public JobExecutionResult executePipeline() throws Exception {
+    final String jobName = options.getJobName();
+
+    if (flinkBatchEnv != null) {
+      return flinkBatchEnv.execute(jobName);
+    } else if (flinkStreamEnv != null) {
+      return flinkStreamEnv.execute(jobName);
+    } else {
+      throw new IllegalStateException("The Pipeline has not yet been translated.");
+    }
+  }
+
+  /**
+   * If the submitted job is a batch processing job, this method creates the adequate
+   * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
+   * on the user-specified options.
+   */
+  private ExecutionEnvironment createBatchExecutionEnvironment() {
+
+    LOG.info("Creating the required Batch Execution Environment.");
+
+    String masterUrl = options.getFlinkMaster();
+    ExecutionEnvironment flinkBatchEnv;
+
+    // depending on the master, create the right environment.
+    if (masterUrl.equals("[local]")) {
+      flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
+    } else if (masterUrl.equals("[collection]")) {
+      flinkBatchEnv = new CollectionEnvironment();
+    } else if (masterUrl.equals("[auto]")) {
+      flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+    } else if (masterUrl.matches(".*:\\d*")) {
+      String[] parts = masterUrl.split(":");
+      List<String> stagingFiles = options.getFilesToStage();
+      flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
+          Integer.parseInt(parts[1]),
+          stagingFiles.toArray(new String[stagingFiles.size()]));
+    } else {
+      LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+      flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+    }
+
+    // set the correct parallelism.
+    if (options.getParallelism() != -1 && !(flinkBatchEnv instanceof CollectionEnvironment)) {
+      flinkBatchEnv.setParallelism(options.getParallelism());
+    }
+
+    // set parallelism in the options (required by some execution code)
+    options.setParallelism(flinkBatchEnv.getParallelism());
+
+    if (options.getObjectReuse()) {
+      flinkBatchEnv.getConfig().enableObjectReuse();
+    } else {
+      flinkBatchEnv.getConfig().disableObjectReuse();
+    }
+
+    return flinkBatchEnv;
+  }
+
+  /**
+   * If the submitted job is a stream processing job, this method creates the adequate
+   * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
+   * on the user-specified options.
+   */
+  private StreamExecutionEnvironment createStreamExecutionEnvironment() {
+
+    LOG.info("Creating the required Streaming Environment.");
+
+    String masterUrl = options.getFlinkMaster();
+    StreamExecutionEnvironment flinkStreamEnv = null;
+
+    // depending on the master, create the right environment.
+    if (masterUrl.equals("[local]")) {
+      flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
+    } else if (masterUrl.equals("[auto]")) {
+      flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+    } else if (masterUrl.matches(".*:\\d*")) {
+      String[] parts = masterUrl.split(":");
+      List<String> stagingFiles = options.getFilesToStage();
+      flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
+          Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
+    } else {
+      LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+      flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+    }
+
+    // set the correct parallelism.
+    if (options.getParallelism() != -1) {
+      flinkStreamEnv.setParallelism(options.getParallelism());
+    }
+
+    // set parallelism in the options (required by some execution code)
+    options.setParallelism(flinkStreamEnv.getParallelism());
+
+    if (options.getObjectReuse()) {
+      flinkStreamEnv.getConfig().enableObjectReuse();
+    } else {
+      flinkStreamEnv.getConfig().disableObjectReuse();
+    }
+
+    // default to event time
+    flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+    // for the following 2 parameters, a value of -1 means that Flink will use
+    // the default values as specified in the configuration.
+    int numRetries = options.getNumberOfExecutionRetries();
+    if (numRetries != -1) {
+      flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
+    }
+    long retryDelay = options.getExecutionRetryDelay();
+    if (retryDelay != -1) {
+      flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
+    }
+
+    // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
+    // If the value is not -1, then the validity checks are applied.
+    // By default, checkpointing is disabled.
+    long checkpointInterval = options.getCheckpointingInterval();
+    if (checkpointInterval != -1) {
+      if (checkpointInterval < 1) {
+        throw new IllegalArgumentException("The checkpoint interval must be positive");
+      }
+      flinkStreamEnv.enableCheckpointing(checkpointInterval);
+    }
+
+    // State backend
+    final AbstractStateBackend stateBackend = options.getStateBackend();
+    if (stateBackend != null) {
+      flinkStreamEnv.setStateBackend(stateBackend);
+    }
+
+    return flinkStreamEnv;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
new file mode 100644
index 0000000..ef9afea
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.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 org.apache.beam.runners.flink;
+
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.List;
+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;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+
+/**
+ * Options which can be used to configure a Flink PipelineRunner.
+ */
+public interface FlinkPipelineOptions
+    extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
+
+  /**
+   * List of local files to make available to workers.
+   *
+   * <p>Jars are placed on the worker's classpath.
+   *
+   * <p>The default value is the list of jars from the main program's classpath.
+   */
+  @Description("Jar-Files to send to all workers and put on the classpath. "
+      + "The default value is all files from the classpath.")
+  @JsonIgnore
+  List<String> getFilesToStage();
+  void setFilesToStage(List<String> value);
+
+  /**
+   * The url of the Flink JobManager on which to execute pipelines. This can either be
+   * the the address of a cluster JobManager, in the form "host:port" or one of the special
+   * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
+   * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
+   * "[auto]" will let the system decide where to execute the pipeline based on the environment.
+   */
+  @Description("Address of the Flink Master where the Pipeline should be executed. Can"
+      + " either be of the form \"host:port\" or one of the special values [local], "
+      + "[collection] or [auto].")
+  String getFlinkMaster();
+  void setFlinkMaster(String value);
+
+  @Description("The degree of parallelism to be used when distributing operations onto workers.")
+  @Default.InstanceFactory(DefaultParallelismFactory.class)
+  Integer getParallelism();
+  void setParallelism(Integer value);
+
+  @Description("The interval between consecutive checkpoints (i.e. snapshots of the current"
+      + "pipeline state used for fault tolerance).")
+  @Default.Long(-1L)
+  Long getCheckpointingInterval();
+  void setCheckpointingInterval(Long interval);
+
+  @Description("Sets the number of times that failed tasks are re-executed. "
+      + "A value of zero effectively disables fault tolerance. A value of -1 indicates "
+      + "that the system default value (as defined in the configuration) should be used.")
+  @Default.Integer(-1)
+  Integer getNumberOfExecutionRetries();
+  void setNumberOfExecutionRetries(Integer retries);
+
+  @Description("Sets the delay between executions. A value of {@code -1} "
+      + "indicates that the default value should be used.")
+  @Default.Long(-1L)
+  Long getExecutionRetryDelay();
+  void setExecutionRetryDelay(Long delay);
+
+  @Description("Sets the behavior of reusing objects.")
+  @Default.Boolean(false)
+  Boolean getObjectReuse();
+  void setObjectReuse(Boolean reuse);
+
+  /**
+   * State backend to store Beam's state during computation.
+   * Note: Only applicable when executing in streaming mode.
+   */
+  @Description("Sets the state backend to use in streaming mode. "
+      + "Otherwise the default is read from the Flink config.")
+  @JsonIgnore
+  AbstractStateBackend getStateBackend();
+  void setStateBackend(AbstractStateBackend stateBackend);
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java
new file mode 100644
index 0000000..65f416d
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.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 org.apache.beam.runners.flink;
+
+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 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.
+ */
+abstract class FlinkPipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+
+  /**
+   * Translates the pipeline by passing this class as a visitor.
+   * @param pipeline The pipeline to be translated
+   */
+  public void translate(Pipeline pipeline) {
+    pipeline.traverseTopologically(this);
+  }
+
+  /**
+   * Utility formatting method.
+   * @param n number of spaces to generate
+   * @return String with "|" followed by n spaces
+   */
+  protected static String genSpaces(int n) {
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < n; i++) {
+      builder.append("|   ");
+    }
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
new file mode 100644
index 0000000..096f030
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import com.google.common.base.Joiner;
+import java.io.File;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+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.TransformHierarchy;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.client.program.DetachedEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to a Flink Plan and then executing them either locally
+ * or on a Flink cluster, depending on the configuration.
+ */
+public class FlinkRunner extends PipelineRunner<PipelineResult> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FlinkRunner.class);
+
+  /**
+   * Provided options.
+   */
+  private final FlinkPipelineOptions options;
+
+  /**
+   * Construct a runner from the provided options.
+   *
+   * @param options Properties which configure the runner.
+   * @return The newly created runner.
+   */
+  public static FlinkRunner fromOptions(PipelineOptions options) {
+    FlinkPipelineOptions flinkOptions =
+        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+    ArrayList<String> missing = new ArrayList<>();
+
+    if (flinkOptions.getAppName() == null) {
+      missing.add("appName");
+    }
+    if (missing.size() > 0) {
+      throw new IllegalArgumentException(
+          "Missing required values: " + Joiner.on(',').join(missing));
+    }
+
+    if (flinkOptions.getFilesToStage() == null) {
+      flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
+          FlinkRunner.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.",
+          flinkOptions.getFilesToStage().size());
+      LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
+    }
+
+    // Set Flink Master to [auto] if no option was specified.
+    if (flinkOptions.getFlinkMaster() == null) {
+      flinkOptions.setFlinkMaster("[auto]");
+    }
+
+    return new FlinkRunner(flinkOptions);
+  }
+
+  private FlinkRunner(FlinkPipelineOptions options) {
+    this.options = options;
+    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
+  }
+
+  @Override
+  public PipelineResult run(Pipeline pipeline) {
+    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
+
+    LOG.info("Executing pipeline using FlinkRunner.");
+
+    FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options);
+
+    LOG.info("Translating pipeline to Flink program.");
+    env.translate(this, pipeline);
+
+    JobExecutionResult result;
+    try {
+      LOG.info("Starting execution of Flink program.");
+      result = env.executePipeline();
+    } catch (Exception e) {
+      LOG.error("Pipeline execution failed", e);
+      throw new RuntimeException("Pipeline execution failed", e);
+    }
+
+    if (result instanceof DetachedEnvironment.DetachedJobExecutionResult) {
+      LOG.info("Pipeline submitted in Detached mode");
+      return new FlinkDetachedRunnerResult();
+    } else {
+      LOG.info("Execution finished in {} msecs", result.getNetRuntime());
+      Map<String, Object> accumulators = result.getAllAccumulatorResults();
+      if (accumulators != null && !accumulators.isEmpty()) {
+        LOG.info("Final aggregator values:");
+
+        for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
+          LOG.info("{} : {}", entry.getKey(), entry.getValue());
+        }
+      }
+
+      return new FlinkRunnerResult(accumulators, result.getNetRuntime());
+    }
+  }
+
+  /**
+   * For testing.
+   */
+  public FlinkPipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  @Override
+  public String toString() {
+    return "FlinkRunner#" + hashCode();
+  }
+
+  /**
+   * Attempts to detect all the resources the class loader has access to. This does not recurse
+   * to class loader parents stopping it from pulling in resources from the system class loader.
+   *
+   * @param classLoader The URLClassLoader to use to detect resources to stage.
+   * @return A list of absolute paths to the resources the class loader uses.
+   * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
+   *   of the resources the class loader exposes is not a file resource.
+   */
+  protected static List<String> detectClassPathResourcesToStage(
+      ClassLoader classLoader) {
+    if (!(classLoader instanceof URLClassLoader)) {
+      String message = String.format("Unable to use ClassLoader to detect classpath elements. "
+          + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
+      LOG.error(message);
+      throw new IllegalArgumentException(message);
+    }
+
+    List<String> files = new ArrayList<>();
+    for (URL url : ((URLClassLoader) classLoader).getURLs()) {
+      try {
+        files.add(new File(url.toURI()).getAbsolutePath());
+      } catch (IllegalArgumentException | URISyntaxException e) {
+        String message = String.format("Unable to convert url (%s) to file.", url);
+        LOG.error(message);
+        throw new IllegalArgumentException(message, e);
+      }
+    }
+    return files;
+  }
+
+  /** A set of {@link View}s with non-deterministic key coders. */
+  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
+
+  /**
+   * Records that the {@link PTransform} requires a deterministic key coder.
+   */
+  void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
+    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
+  }
+
+  /** 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 Pipeline.PipelineVisitor() {
+        @Override
+        public void visitValue(PValue value, TransformHierarchy.Node producer) {
+        }
+
+        @Override
+        public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+        }
+
+        @Override
+        public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+          return CompositeBehavior.ENTER_TRANSFORM;
+        }
+
+        @Override
+        public void leaveCompositeTransform(TransformHierarchy.Node 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 "
+          + "the Flink runner will require deterministic key coders.",
+          ptransformViewNamesWithNonDeterministicKeyCoders);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
new file mode 100644
index 0000000..681459a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.flink;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+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;
+
+
+/**
+ * AutoService registrar - will register FlinkRunner and FlinkOptions
+ * as possible pipeline runner services.
+ *
+ * <p>It ends up in META-INF/services and gets picked up by Beam.
+ *
+ */
+public class FlinkRunnerRegistrar {
+  private FlinkRunnerRegistrar() { }
+
+  /**
+   * Pipeline runner registrar.
+   */
+  @AutoService(PipelineRunnerRegistrar.class)
+  public static class Runner implements PipelineRunnerRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
+          FlinkRunner.class,
+          TestFlinkRunner.class);
+    }
+  }
+
+  /**
+   * Pipeline options registrar.
+   */
+  @AutoService(PipelineOptionsRegistrar.class)
+  public static class Options implements PipelineOptionsRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+      return ImmutableList.<Class<? extends PipelineOptions>>of(FlinkPipelineOptions.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
new file mode 100644
index 0000000..0682b56
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.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 org.apache.beam.runners.flink;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.beam.sdk.AggregatorRetrievalException;
+import org.apache.beam.sdk.AggregatorValues;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.MetricResults;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.joda.time.Duration;
+
+/**
+ * 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 org.apache.beam.sdk.transforms.Aggregator}s.
+ */
+public class FlinkRunnerResult implements PipelineResult {
+
+  private final Map<String, Object> aggregators;
+
+  private final long runtime;
+
+  FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
+    this.aggregators = (aggregators == null || aggregators.isEmpty())
+        ? Collections.<String, Object>emptyMap()
+        : Collections.unmodifiableMap(aggregators);
+    this.runtime = runtime;
+  }
+
+  @Override
+  public State getState() {
+    return State.DONE;
+  }
+
+  @Override
+  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
+      throws AggregatorRetrievalException {
+    // TODO provide a list of all accumulator step values
+    Object value = aggregators.get(aggregator.getName());
+    if (value != null) {
+      return new AggregatorValues<T>() {
+        @Override
+        public Map<String, T> getValuesAtSteps() {
+          return (Map<String, T>) aggregators;
+        }
+      };
+    } else {
+      throw new AggregatorRetrievalException("Accumulator results not found.",
+          new RuntimeException("Accumulator does not exist."));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "FlinkRunnerResult{"
+        + "aggregators=" + aggregators
+        + ", runtime=" + runtime
+        + '}';
+  }
+
+  @Override
+  public State cancel() throws IOException {
+    throw new UnsupportedOperationException("FlinkRunnerResult does not support cancel.");
+  }
+
+  @Override
+  public State waitUntilFinish() {
+    return State.DONE;
+  }
+
+  @Override
+  public State waitUntilFinish(Duration duration) {
+    return State.DONE;
+  }
+
+  @Override
+  public MetricResults metrics() {
+    throw new UnsupportedOperationException("The FlinkRunner does not currently support metrics.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
new file mode 100644
index 0000000..0459ef7
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.core.construction.PTransformMatchers;
+import org.apache.beam.runners.core.construction.PTransformReplacements;
+import org.apache.beam.runners.core.construction.ReplacementOutputs;
+import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PTransformOverride;
+import org.apache.beam.sdk.runners.PTransformOverrideFactory;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+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 org.apache.beam.sdk.values.PCollection}-based job into a
+ * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
+ *
+ */
+class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FlinkStreamingPipelineTranslator.class);
+
+  /** The necessary context in the case of a straming job. */
+  private final FlinkStreamingTranslationContext streamingContext;
+
+  private int depth = 0;
+
+  private FlinkRunner flinkRunner;
+
+  public FlinkStreamingPipelineTranslator(
+      FlinkRunner flinkRunner,
+      StreamExecutionEnvironment env,
+      PipelineOptions options) {
+    this.streamingContext = new FlinkStreamingTranslationContext(env, options);
+    this.flinkRunner = flinkRunner;
+  }
+
+  @Override
+  public void translate(Pipeline pipeline) {
+    List<PTransformOverride> transformOverrides =
+        ImmutableList.<PTransformOverride>builder()
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.splittableParDoMulti(),
+                    new SplittableParDoOverrideFactory()))
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(View.AsIterable.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingViewAsIterable.class, flinkRunner)))
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(View.AsList.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingViewAsList.class, flinkRunner)))
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(View.AsMap.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingViewAsMap.class, flinkRunner)))
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(View.AsMultimap.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingViewAsMultimap.class, flinkRunner)))
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(View.AsSingleton.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingViewAsSingleton.class, flinkRunner)))
+            // this has to be last since the ViewAsSingleton override
+            // can expand to a Combine.GloballyAsSingletonView
+            .add(
+                PTransformOverride.of(
+                    PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class),
+                    new ReflectiveOneToOneOverrideFactory(
+                        FlinkStreamingViewOverrides.StreamingCombineGloballyAsSingletonView.class,
+                        flinkRunner)))
+            .build();
+
+    pipeline.replaceAll(transformOverrides);
+    super.translate(pipeline);
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Pipeline Visitor Methods
+  // --------------------------------------------------------------------------------------------
+
+  @Override
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
+    LOG.info("{} enterCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
+    this.depth++;
+
+    PTransform<?, ?> transform = node.getTransform();
+    if (transform != null) {
+      StreamTransformTranslator<?> translator =
+          FlinkStreamingTransformTranslators.getTranslator(transform);
+
+      if (translator != null && applyCanTranslate(transform, node, translator)) {
+        applyStreamingTransform(transform, node, translator);
+        LOG.info("{} translated- {}", genSpaces(this.depth), node.getFullName());
+        return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+      }
+    }
+    return CompositeBehavior.ENTER_TRANSFORM;
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
+    this.depth--;
+    LOG.info("{} leaveCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
+  }
+
+  @Override
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+    LOG.info("{} visitPrimitiveTransform- {}", genSpaces(this.depth), node.getFullName());
+    // get the transformation corresponding to hte node we are
+    // currently visiting and translate it into its Flink alternative.
+
+    PTransform<?, ?> transform = node.getTransform();
+    StreamTransformTranslator<?> translator =
+        FlinkStreamingTransformTranslators.getTranslator(transform);
+
+    if (translator == null || !applyCanTranslate(transform, node, translator)) {
+      LOG.info(node.getTransform().getClass().toString());
+      throw new UnsupportedOperationException(
+          "The transform " + transform + " is currently not supported.");
+    }
+    applyStreamingTransform(transform, node, translator);
+  }
+
+  @Override
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
+    // do nothing here
+  }
+
+  private <T extends PTransform<?, ?>> void applyStreamingTransform(
+      PTransform<?, ?> transform,
+      TransformHierarchy.Node node,
+      StreamTransformTranslator<?> translator) {
+
+    @SuppressWarnings("unchecked")
+    T typedTransform = (T) transform;
+
+    @SuppressWarnings("unchecked")
+    StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
+
+    // create the applied PTransform on the streamingContext
+    streamingContext.setCurrentTransform(node.toAppliedPTransform());
+    typedTranslator.translateNode(typedTransform, streamingContext);
+  }
+
+  private <T extends PTransform<?, ?>> boolean applyCanTranslate(
+      PTransform<?, ?> transform,
+      TransformHierarchy.Node node,
+      StreamTransformTranslator<?> translator) {
+
+    @SuppressWarnings("unchecked")
+    T typedTransform = (T) transform;
+
+    @SuppressWarnings("unchecked")
+    StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
+
+    streamingContext.setCurrentTransform(node.toAppliedPTransform());
+
+    return typedTranslator.canTranslate(typedTransform, streamingContext);
+  }
+
+  /**
+   * The interface that every Flink translator of a Beam operator should implement.
+   * This interface is for <b>streaming</b> jobs. For examples of such translators see
+   * {@link FlinkStreamingTransformTranslators}.
+   */
+  abstract static class StreamTransformTranslator<T extends PTransform> {
+
+    /**
+     * Translate the given transform.
+     */
+    abstract void translateNode(T transform, FlinkStreamingTranslationContext context);
+
+    /**
+     * Returns true iff this translator can translate the given transform.
+     */
+    boolean canTranslate(T transform, FlinkStreamingTranslationContext context) {
+      return true;
+    }
+  }
+
+  private static class ReflectiveOneToOneOverrideFactory<
+          InputT, OutputT, TransformT extends PTransform<PCollection<InputT>, PCollection<OutputT>>>
+      extends SingleInputOutputOverrideFactory<
+          PCollection<InputT>, PCollection<OutputT>, TransformT> {
+    private final Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement;
+    private final FlinkRunner runner;
+
+    private ReflectiveOneToOneOverrideFactory(
+        Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement,
+        FlinkRunner runner) {
+      this.replacement = replacement;
+      this.runner = runner;
+    }
+
+    @Override
+    public PTransformReplacement<PCollection<InputT>, PCollection<OutputT>> getReplacementTransform(
+        AppliedPTransform<PCollection<InputT>, PCollection<OutputT>, TransformT> transform) {
+      return PTransformReplacement.of(
+          PTransformReplacements.getSingletonMainInput(transform),
+          InstanceBuilder.ofType(replacement)
+              .withArg(FlinkRunner.class, runner)
+              .withArg(
+                  (Class<PTransform<PCollection<InputT>, PCollection<OutputT>>>)
+                      transform.getTransform().getClass(),
+                  transform.getTransform())
+              .build());
+    }
+  }
+
+  /**
+   * A {@link PTransformOverrideFactory} that overrides a <a
+   * href="https://s.apache.org/splittable-do-fn">Splittable DoFn</a> with {@link SplittableParDo}.
+   */
+  static class SplittableParDoOverrideFactory<InputT, OutputT>
+      implements PTransformOverrideFactory<
+          PCollection<InputT>, PCollectionTuple, MultiOutput<InputT, OutputT>> {
+    @Override
+    public PTransformReplacement<PCollection<InputT>, PCollectionTuple>
+        getReplacementTransform(
+            AppliedPTransform<
+                    PCollection<InputT>, PCollectionTuple, MultiOutput<InputT, OutputT>>
+                transform) {
+      return PTransformReplacement.of(
+          PTransformReplacements.getSingletonMainInput(transform),
+          new SplittableParDo<>(transform.getTransform()));
+    }
+
+    @Override
+    public Map<PValue, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+      return ReplacementOutputs.tagged(outputs, newOutput);
+    }
+  }
+}


[15/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 681459a..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.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 org.apache.beam.runners.flink;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-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;
-
-
-/**
- * AutoService registrar - will register FlinkRunner and FlinkOptions
- * as possible pipeline runner services.
- *
- * <p>It ends up in META-INF/services and gets picked up by Beam.
- *
- */
-public class FlinkRunnerRegistrar {
-  private FlinkRunnerRegistrar() { }
-
-  /**
-   * Pipeline runner registrar.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class Runner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
-          FlinkRunner.class,
-          TestFlinkRunner.class);
-    }
-  }
-
-  /**
-   * Pipeline options registrar.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(FlinkPipelineOptions.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 0682b56..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.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 org.apache.beam.runners.flink;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import org.apache.beam.sdk.AggregatorRetrievalException;
-import org.apache.beam.sdk.AggregatorValues;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.joda.time.Duration;
-
-/**
- * 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 org.apache.beam.sdk.transforms.Aggregator}s.
- */
-public class FlinkRunnerResult implements PipelineResult {
-
-  private final Map<String, Object> aggregators;
-
-  private final long runtime;
-
-  FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
-    this.aggregators = (aggregators == null || aggregators.isEmpty())
-        ? Collections.<String, Object>emptyMap()
-        : Collections.unmodifiableMap(aggregators);
-    this.runtime = runtime;
-  }
-
-  @Override
-  public State getState() {
-    return State.DONE;
-  }
-
-  @Override
-  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
-      throws AggregatorRetrievalException {
-    // TODO provide a list of all accumulator step values
-    Object value = aggregators.get(aggregator.getName());
-    if (value != null) {
-      return new AggregatorValues<T>() {
-        @Override
-        public Map<String, T> getValuesAtSteps() {
-          return (Map<String, T>) aggregators;
-        }
-      };
-    } else {
-      throw new AggregatorRetrievalException("Accumulator results not found.",
-          new RuntimeException("Accumulator does not exist."));
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "FlinkRunnerResult{"
-        + "aggregators=" + aggregators
-        + ", runtime=" + runtime
-        + '}';
-  }
-
-  @Override
-  public State cancel() throws IOException {
-    throw new UnsupportedOperationException("FlinkRunnerResult does not support cancel.");
-  }
-
-  @Override
-  public State waitUntilFinish() {
-    return State.DONE;
-  }
-
-  @Override
-  public State waitUntilFinish(Duration duration) {
-    return State.DONE;
-  }
-
-  @Override
-  public MetricResults metrics() {
-    throw new UnsupportedOperationException("The FlinkRunner does not currently support metrics.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
deleted file mode 100644
index 0459ef7..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import com.google.common.collect.ImmutableList;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.SplittableParDo;
-import org.apache.beam.runners.core.construction.PTransformMatchers;
-import org.apache.beam.runners.core.construction.PTransformReplacements;
-import org.apache.beam.runners.core.construction.ReplacementOutputs;
-import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PTransformOverride;
-import org.apache.beam.sdk.runners.PTransformOverrideFactory;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TupleTag;
-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 org.apache.beam.sdk.values.PCollection}-based job into a
- * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
- *
- */
-class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
-
-  private static final Logger LOG = LoggerFactory.getLogger(FlinkStreamingPipelineTranslator.class);
-
-  /** The necessary context in the case of a straming job. */
-  private final FlinkStreamingTranslationContext streamingContext;
-
-  private int depth = 0;
-
-  private FlinkRunner flinkRunner;
-
-  public FlinkStreamingPipelineTranslator(
-      FlinkRunner flinkRunner,
-      StreamExecutionEnvironment env,
-      PipelineOptions options) {
-    this.streamingContext = new FlinkStreamingTranslationContext(env, options);
-    this.flinkRunner = flinkRunner;
-  }
-
-  @Override
-  public void translate(Pipeline pipeline) {
-    List<PTransformOverride> transformOverrides =
-        ImmutableList.<PTransformOverride>builder()
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.splittableParDoMulti(),
-                    new SplittableParDoOverrideFactory()))
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(View.AsIterable.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingViewAsIterable.class, flinkRunner)))
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(View.AsList.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingViewAsList.class, flinkRunner)))
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(View.AsMap.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingViewAsMap.class, flinkRunner)))
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(View.AsMultimap.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingViewAsMultimap.class, flinkRunner)))
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(View.AsSingleton.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingViewAsSingleton.class, flinkRunner)))
-            // this has to be last since the ViewAsSingleton override
-            // can expand to a Combine.GloballyAsSingletonView
-            .add(
-                PTransformOverride.of(
-                    PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class),
-                    new ReflectiveOneToOneOverrideFactory(
-                        FlinkStreamingViewOverrides.StreamingCombineGloballyAsSingletonView.class,
-                        flinkRunner)))
-            .build();
-
-    pipeline.replaceAll(transformOverrides);
-    super.translate(pipeline);
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline Visitor Methods
-  // --------------------------------------------------------------------------------------------
-
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    LOG.info("{} enterCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
-    this.depth++;
-
-    PTransform<?, ?> transform = node.getTransform();
-    if (transform != null) {
-      StreamTransformTranslator<?> translator =
-          FlinkStreamingTransformTranslators.getTranslator(transform);
-
-      if (translator != null && applyCanTranslate(transform, node, translator)) {
-        applyStreamingTransform(transform, node, translator);
-        LOG.info("{} translated- {}", genSpaces(this.depth), node.getFullName());
-        return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
-      }
-    }
-    return CompositeBehavior.ENTER_TRANSFORM;
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformHierarchy.Node node) {
-    this.depth--;
-    LOG.info("{} leaveCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
-  }
-
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    LOG.info("{} visitPrimitiveTransform- {}", genSpaces(this.depth), node.getFullName());
-    // get the transformation corresponding to hte node we are
-    // currently visiting and translate it into its Flink alternative.
-
-    PTransform<?, ?> transform = node.getTransform();
-    StreamTransformTranslator<?> translator =
-        FlinkStreamingTransformTranslators.getTranslator(transform);
-
-    if (translator == null || !applyCanTranslate(transform, node, translator)) {
-      LOG.info(node.getTransform().getClass().toString());
-      throw new UnsupportedOperationException(
-          "The transform " + transform + " is currently not supported.");
-    }
-    applyStreamingTransform(transform, node, translator);
-  }
-
-  @Override
-  public void visitValue(PValue value, TransformHierarchy.Node producer) {
-    // do nothing here
-  }
-
-  private <T extends PTransform<?, ?>> void applyStreamingTransform(
-      PTransform<?, ?> transform,
-      TransformHierarchy.Node node,
-      StreamTransformTranslator<?> translator) {
-
-    @SuppressWarnings("unchecked")
-    T typedTransform = (T) transform;
-
-    @SuppressWarnings("unchecked")
-    StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
-
-    // create the applied PTransform on the streamingContext
-    streamingContext.setCurrentTransform(node.toAppliedPTransform());
-    typedTranslator.translateNode(typedTransform, streamingContext);
-  }
-
-  private <T extends PTransform<?, ?>> boolean applyCanTranslate(
-      PTransform<?, ?> transform,
-      TransformHierarchy.Node node,
-      StreamTransformTranslator<?> translator) {
-
-    @SuppressWarnings("unchecked")
-    T typedTransform = (T) transform;
-
-    @SuppressWarnings("unchecked")
-    StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
-
-    streamingContext.setCurrentTransform(node.toAppliedPTransform());
-
-    return typedTranslator.canTranslate(typedTransform, streamingContext);
-  }
-
-  /**
-   * The interface that every Flink translator of a Beam operator should implement.
-   * This interface is for <b>streaming</b> jobs. For examples of such translators see
-   * {@link FlinkStreamingTransformTranslators}.
-   */
-  abstract static class StreamTransformTranslator<T extends PTransform> {
-
-    /**
-     * Translate the given transform.
-     */
-    abstract void translateNode(T transform, FlinkStreamingTranslationContext context);
-
-    /**
-     * Returns true iff this translator can translate the given transform.
-     */
-    boolean canTranslate(T transform, FlinkStreamingTranslationContext context) {
-      return true;
-    }
-  }
-
-  private static class ReflectiveOneToOneOverrideFactory<
-          InputT, OutputT, TransformT extends PTransform<PCollection<InputT>, PCollection<OutputT>>>
-      extends SingleInputOutputOverrideFactory<
-          PCollection<InputT>, PCollection<OutputT>, TransformT> {
-    private final Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement;
-    private final FlinkRunner runner;
-
-    private ReflectiveOneToOneOverrideFactory(
-        Class<PTransform<PCollection<InputT>, PCollection<OutputT>>> replacement,
-        FlinkRunner runner) {
-      this.replacement = replacement;
-      this.runner = runner;
-    }
-
-    @Override
-    public PTransformReplacement<PCollection<InputT>, PCollection<OutputT>> getReplacementTransform(
-        AppliedPTransform<PCollection<InputT>, PCollection<OutputT>, TransformT> transform) {
-      return PTransformReplacement.of(
-          PTransformReplacements.getSingletonMainInput(transform),
-          InstanceBuilder.ofType(replacement)
-              .withArg(FlinkRunner.class, runner)
-              .withArg(
-                  (Class<PTransform<PCollection<InputT>, PCollection<OutputT>>>)
-                      transform.getTransform().getClass(),
-                  transform.getTransform())
-              .build());
-    }
-  }
-
-  /**
-   * A {@link PTransformOverrideFactory} that overrides a <a
-   * href="https://s.apache.org/splittable-do-fn">Splittable DoFn</a> with {@link SplittableParDo}.
-   */
-  static class SplittableParDoOverrideFactory<InputT, OutputT>
-      implements PTransformOverrideFactory<
-          PCollection<InputT>, PCollectionTuple, MultiOutput<InputT, OutputT>> {
-    @Override
-    public PTransformReplacement<PCollection<InputT>, PCollectionTuple>
-        getReplacementTransform(
-            AppliedPTransform<
-                    PCollection<InputT>, PCollectionTuple, MultiOutput<InputT, OutputT>>
-                transform) {
-      return PTransformReplacement.of(
-          PTransformReplacements.getSingletonMainInput(transform),
-          new SplittableParDo<>(transform.getTransform()));
-    }
-
-    @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
-      return ReplacementOutputs.tagged(outputs, newOutput);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
deleted file mode 100644
index 123d5e7..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
+++ /dev/null
@@ -1,1044 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.flink;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.beam.runners.core.ElementAndRestriction;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.SplittableParDo;
-import org.apache.beam.runners.core.SystemReduceFn;
-import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.KvToByteBufferKeySelector;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.SingletonKeyedWorkItem;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.SingletonKeyedWorkItemCoder;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.SplittableDoFnOperator;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.WindowDoFnOperator;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.WorkItemKeySelector;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.BoundedSourceWrapper;
-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.coders.StringUtf8Coder;
-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.transforms.Combine;
-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.join.UnionCoder;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
-import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-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.Reshuffle;
-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 org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
-import org.apache.flink.util.Collector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class contains all the mappings between Beam and Flink
- * <b>streaming</b> transformations. The {@link FlinkStreamingPipelineTranslator}
- * traverses the Beam job and comes here to translate the encountered Beam transformations
- * into Flink one, based on the mapping available in this class.
- */
-class FlinkStreamingTransformTranslators {
-
-  // --------------------------------------------------------------------------------------------
-  //  Transform Translator Registry
-  // --------------------------------------------------------------------------------------------
-
-  @SuppressWarnings("rawtypes")
-  private static final Map<
-      Class<? extends PTransform>,
-      FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
-
-  // here you can find all the available translators.
-  static {
-    TRANSLATORS.put(Read.Bounded.class, new BoundedReadSourceTranslator());
-    TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
-    TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
-
-    TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoStreamingTranslator());
-    TRANSLATORS.put(
-        SplittableParDo.ProcessElements.class, new SplittableProcessElementsStreamingTranslator());
-    TRANSLATORS.put(
-        SplittableParDo.GBKIntoKeyedWorkItems.class, new GBKIntoKeyedWorkItemsTranslator());
-
-
-    TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator());
-    TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslator());
-    TRANSLATORS.put(
-        FlinkStreamingViewOverrides.CreateFlinkPCollectionView.class,
-        new CreateViewStreamingTranslator());
-
-    TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorStreaming());
-    TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
-    TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
-  }
-
-  public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(
-      PTransform<?, ?> transform) {
-    return TRANSLATORS.get(transform.getClass());
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Transformation Implementations
-  // --------------------------------------------------------------------------------------------
-
-  private static class TextIOWriteBoundStreamingTranslator
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound> {
-
-    private static final Logger LOG =
-        LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
-
-    @Override
-    public void translateNode(
-        TextIO.Write.Bound transform,
-        FlinkStreamingTranslationContext context) {
-      PValue input = context.getInput(transform);
-      DataStream<WindowedValue<String>> inputDataStream = context.getInputDataStream(input);
-
-      String filenamePrefix = transform.getFilenamePrefix();
-      String filenameSuffix = transform.getFilenameSuffix();
-      boolean needsValidation = transform.needsValidation();
-      int numShards = transform.getNumShards();
-      String shardNameTemplate = transform.getShardNameTemplate();
-
-      // TODO: Implement these. We need Flink support for this.
-      LOG.warn(
-          "Translation of TextIO.Write.needsValidation not yet supported. Is: {}.",
-          needsValidation);
-      LOG.warn(
-          "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
-          filenameSuffix);
-      LOG.warn(
-          "Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.",
-          shardNameTemplate);
-
-      DataStream<String> dataSink = inputDataStream
-          .flatMap(new FlatMapFunction<WindowedValue<String>, String>() {
-            @Override
-            public void flatMap(
-                WindowedValue<String> value,
-                Collector<String> out)
-                throws Exception {
-              out.collect(value.getValue());
-            }
-          });
-      DataStreamSink<String> output =
-          dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
-
-      if (numShards > 0) {
-        output.setParallelism(numShards);
-      }
-    }
-  }
-
-  private static class UnboundedReadSourceTranslator<T>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
-
-    @Override
-    public void translateNode(
-        Read.Unbounded<T> transform,
-        FlinkStreamingTranslationContext context) {
-      PCollection<T> output = context.getOutput(transform);
-
-      TypeInformation<WindowedValue<T>> outputTypeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-      DataStream<WindowedValue<T>> source;
-      try {
-        UnboundedSourceWrapper<T, ?> sourceWrapper =
-            new UnboundedSourceWrapper<>(
-                context.getPipelineOptions(),
-                transform.getSource(),
-                context.getExecutionEnvironment().getParallelism());
-        source = context
-            .getExecutionEnvironment()
-            .addSource(sourceWrapper).name(transform.getName()).returns(outputTypeInfo);
-      } catch (Exception e) {
-        throw new RuntimeException(
-            "Error while translating UnboundedSource: " + transform.getSource(), e);
-      }
-
-      context.setOutputDataStream(output, source);
-    }
-  }
-
-  private static class BoundedReadSourceTranslator<T>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Bounded<T>> {
-
-    @Override
-    public void translateNode(
-        Read.Bounded<T> transform,
-        FlinkStreamingTranslationContext context) {
-      PCollection<T> output = context.getOutput(transform);
-
-      TypeInformation<WindowedValue<T>> outputTypeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-
-      DataStream<WindowedValue<T>> source;
-      try {
-        BoundedSourceWrapper<T> sourceWrapper =
-            new BoundedSourceWrapper<>(
-                context.getPipelineOptions(),
-                transform.getSource(),
-                context.getExecutionEnvironment().getParallelism());
-        source = context
-            .getExecutionEnvironment()
-            .addSource(sourceWrapper).name(transform.getName()).returns(outputTypeInfo);
-      } catch (Exception e) {
-        throw new RuntimeException(
-            "Error while translating BoundedSource: " + transform.getSource(), e);
-      }
-
-      context.setOutputDataStream(output, source);
-    }
-  }
-
-  /**
-   * Wraps each element in a {@link RawUnionValue} with the given tag id.
-   */
-  private static class ToRawUnion<T> implements MapFunction<T, RawUnionValue> {
-    private final int intTag;
-
-    public ToRawUnion(int intTag) {
-      this.intTag = intTag;
-    }
-
-    @Override
-    public RawUnionValue map(T o) throws Exception {
-      return new RawUnionValue(intTag, o);
-    }
-  }
-
-  private static Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>>
-        transformSideInputs(
-          Collection<PCollectionView<?>> sideInputs,
-          FlinkStreamingTranslationContext context) {
-
-    // collect all side inputs
-    Map<TupleTag<?>, Integer> tagToIntMapping = new HashMap<>();
-    Map<Integer, PCollectionView<?>> intToViewMapping = new HashMap<>();
-    int count = 0;
-    for (PCollectionView<?> sideInput: sideInputs) {
-      TupleTag<?> tag = sideInput.getTagInternal();
-      intToViewMapping.put(count, sideInput);
-      tagToIntMapping.put(tag, count);
-      count++;
-      Coder<Iterable<WindowedValue<?>>> coder = sideInput.getCoderInternal();
-    }
-
-
-    List<Coder<?>> inputCoders = new ArrayList<>();
-    for (PCollectionView<?> sideInput: sideInputs) {
-      DataStream<Object> sideInputStream = context.getInputDataStream(sideInput);
-      TypeInformation<Object> tpe = sideInputStream.getType();
-      if (!(tpe instanceof CoderTypeInformation)) {
-        throw new IllegalStateException(
-            "Input Stream TypeInformation is no CoderTypeInformation.");
-      }
-
-      Coder<?> coder = ((CoderTypeInformation) tpe).getCoder();
-      inputCoders.add(coder);
-    }
-
-    UnionCoder unionCoder = UnionCoder.of(inputCoders);
-
-    CoderTypeInformation<RawUnionValue> unionTypeInformation =
-        new CoderTypeInformation<>(unionCoder);
-
-    // transform each side input to RawUnionValue and union them
-    DataStream<RawUnionValue> sideInputUnion = null;
-
-    for (PCollectionView<?> sideInput: sideInputs) {
-      TupleTag<?> tag = sideInput.getTagInternal();
-      final int intTag = tagToIntMapping.get(tag);
-      DataStream<Object> sideInputStream = context.getInputDataStream(sideInput);
-      DataStream<RawUnionValue> unionValueStream =
-          sideInputStream.map(new ToRawUnion<>(intTag)).returns(unionTypeInformation);
-
-      if (sideInputUnion == null) {
-        sideInputUnion = unionValueStream;
-      } else {
-        sideInputUnion = sideInputUnion.union(unionValueStream);
-      }
-    }
-
-    if (sideInputUnion == null) {
-      throw new IllegalStateException("No unioned side inputs, this indicates a bug.");
-    }
-
-    return new Tuple2<>(intToViewMapping, sideInputUnion);
-  }
-
-  /**
-   * Helper for translating {@link ParDo.MultiOutput} and {@link SplittableParDo.ProcessElements}.
-   */
-  static class ParDoTranslationHelper {
-
-    interface DoFnOperatorFactory<InputT, OutputT> {
-      DoFnOperator<InputT, OutputT, RawUnionValue> createDoFnOperator(
-          DoFn<InputT, OutputT> doFn,
-          List<PCollectionView<?>> sideInputs,
-          TupleTag<OutputT> mainOutputTag,
-          List<TupleTag<?>> additionalOutputTags,
-          FlinkStreamingTranslationContext context,
-          WindowingStrategy<?, ?> windowingStrategy,
-          Map<TupleTag<?>, Integer> tagsToLabels,
-          Coder<WindowedValue<InputT>> inputCoder,
-          Coder keyCoder,
-          Map<Integer, PCollectionView<?>> transformedSideInputs);
-    }
-
-    static <InputT, OutputT> void translateParDo(
-        String transformName,
-        DoFn<InputT, OutputT> doFn,
-        PCollection<InputT> input,
-        List<PCollectionView<?>> sideInputs,
-        Map<TupleTag<?>, PValue> outputs,
-        TupleTag<OutputT> mainOutputTag,
-        List<TupleTag<?>> additionalOutputTags,
-        FlinkStreamingTranslationContext context,
-        DoFnOperatorFactory<InputT, OutputT> doFnOperatorFactory) {
-
-      // we assume that the transformation does not change the windowing strategy.
-      WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
-      Map<TupleTag<?>, Integer> tagsToLabels =
-          transformTupleTagsToLabels(mainOutputTag, outputs);
-
-      SingleOutputStreamOperator<RawUnionValue> unionOutputStream;
-
-      Coder<WindowedValue<InputT>> inputCoder = context.getCoder(input);
-
-      DataStream<WindowedValue<InputT>> inputDataStream = context.getInputDataStream(input);
-
-      Coder keyCoder = null;
-      boolean stateful = false;
-      DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
-      if (signature.stateDeclarations().size() > 0
-          || signature.timerDeclarations().size() > 0) {
-        // Based on the fact that the signature is stateful, DoFnSignatures ensures
-        // that it is also keyed
-        keyCoder = ((KvCoder) input.getCoder()).getKeyCoder();
-        inputDataStream = inputDataStream.keyBy(new KvToByteBufferKeySelector(keyCoder));
-        stateful = true;
-      } else if (doFn instanceof SplittableParDo.ProcessFn) {
-        // we know that it is keyed on String
-        keyCoder = StringUtf8Coder.of();
-        stateful = true;
-      }
-
-      if (sideInputs.isEmpty()) {
-        DoFnOperator<InputT, OutputT, RawUnionValue> doFnOperator =
-            doFnOperatorFactory.createDoFnOperator(
-                doFn,
-                sideInputs,
-                mainOutputTag,
-                additionalOutputTags,
-                context,
-                windowingStrategy,
-                tagsToLabels,
-                inputCoder,
-                keyCoder,
-                new HashMap<Integer, PCollectionView<?>>() /* side-input mapping */);
-
-        UnionCoder outputUnionCoder = createUnionCoder(outputs);
-
-        CoderTypeInformation<RawUnionValue> outputUnionTypeInformation =
-            new CoderTypeInformation<>(outputUnionCoder);
-
-        unionOutputStream = inputDataStream
-            .transform(transformName, outputUnionTypeInformation, doFnOperator);
-
-      } else {
-        Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>> transformedSideInputs =
-            transformSideInputs(sideInputs, context);
-
-        DoFnOperator<InputT, OutputT, RawUnionValue> doFnOperator =
-            doFnOperatorFactory.createDoFnOperator(
-                doFn,
-                sideInputs,
-                mainOutputTag,
-                additionalOutputTags,
-                context,
-                windowingStrategy,
-                tagsToLabels,
-                inputCoder,
-                keyCoder,
-                transformedSideInputs.f0);
-
-        UnionCoder outputUnionCoder = createUnionCoder(outputs);
-
-        CoderTypeInformation<RawUnionValue> outputUnionTypeInformation =
-            new CoderTypeInformation<>(outputUnionCoder);
-
-        if (stateful) {
-          // we have to manually contruct the two-input transform because we're not
-          // allowed to have only one input keyed, normally.
-          KeyedStream keyedStream = (KeyedStream<?, InputT>) inputDataStream;
-          TwoInputTransformation<
-              WindowedValue<KV<?, InputT>>,
-              RawUnionValue,
-              WindowedValue<OutputT>> rawFlinkTransform = new TwoInputTransformation(
-              keyedStream.getTransformation(),
-              transformedSideInputs.f1.broadcast().getTransformation(),
-              transformName,
-              (TwoInputStreamOperator) doFnOperator,
-              outputUnionTypeInformation,
-              keyedStream.getParallelism());
-
-          rawFlinkTransform.setStateKeyType(keyedStream.getKeyType());
-          rawFlinkTransform.setStateKeySelectors(keyedStream.getKeySelector(), null);
-
-          unionOutputStream = new SingleOutputStreamOperator(
-                  keyedStream.getExecutionEnvironment(),
-                  rawFlinkTransform) {}; // we have to cheat around the ctor being protected
-
-          keyedStream.getExecutionEnvironment().addOperator(rawFlinkTransform);
-
-        } else {
-          unionOutputStream = inputDataStream
-              .connect(transformedSideInputs.f1.broadcast())
-              .transform(transformName, outputUnionTypeInformation, doFnOperator);
-        }
-      }
-
-      SplitStream<RawUnionValue> splitStream = unionOutputStream
-              .split(new OutputSelector<RawUnionValue>() {
-                @Override
-                public Iterable<String> select(RawUnionValue value) {
-                  return Collections.singletonList(Integer.toString(value.getUnionTag()));
-                }
-              });
-
-      for (Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
-        final int outputTag = tagsToLabels.get(output.getKey());
-
-        TypeInformation outputTypeInfo = context.getTypeInfo((PCollection<?>) output.getValue());
-
-        @SuppressWarnings("unchecked")
-        DataStream unwrapped = splitStream.select(String.valueOf(outputTag))
-          .flatMap(new FlatMapFunction<RawUnionValue, Object>() {
-            @Override
-            public void flatMap(RawUnionValue value, Collector<Object> out) throws Exception {
-              out.collect(value.getValue());
-            }
-          }).returns(outputTypeInfo);
-
-        context.setOutputDataStream(output.getValue(), unwrapped);
-      }
-    }
-
-    private static Map<TupleTag<?>, Integer> transformTupleTagsToLabels(
-        TupleTag<?> mainTag,
-        Map<TupleTag<?>, PValue> allTaggedValues) {
-
-      Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
-      int count = 0;
-      tagToLabelMap.put(mainTag, count++);
-      for (TupleTag<?> key : allTaggedValues.keySet()) {
-        if (!tagToLabelMap.containsKey(key)) {
-          tagToLabelMap.put(key, count++);
-        }
-      }
-      return tagToLabelMap;
-    }
-
-    private static UnionCoder createUnionCoder(Map<TupleTag<?>, PValue> taggedCollections) {
-      List<Coder<?>> outputCoders = Lists.newArrayList();
-      for (PValue taggedColl : taggedCollections.values()) {
-        checkArgument(
-            taggedColl instanceof PCollection,
-            "A Union Coder can only be created for a Collection of Tagged %s. Got %s",
-            PCollection.class.getSimpleName(),
-            taggedColl.getClass().getSimpleName());
-        PCollection<?> coll = (PCollection<?>) taggedColl;
-        WindowedValue.FullWindowedValueCoder<?> windowedValueCoder =
-            WindowedValue.getFullCoder(
-                coll.getCoder(),
-                coll.getWindowingStrategy().getWindowFn().windowCoder());
-        outputCoders.add(windowedValueCoder);
-      }
-      return UnionCoder.of(outputCoders);
-    }
-  }
-
-  private static class ParDoStreamingTranslator<InputT, OutputT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      ParDo.MultiOutput<InputT, OutputT>> {
-
-    @Override
-    public void translateNode(
-        ParDo.MultiOutput<InputT, OutputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      ParDoTranslationHelper.translateParDo(
-          transform.getName(),
-          transform.getFn(),
-          (PCollection<InputT>) context.getInput(transform),
-          transform.getSideInputs(),
-          context.getOutputs(transform),
-          transform.getMainOutputTag(),
-          transform.getAdditionalOutputTags().getAll(),
-          context,
-          new ParDoTranslationHelper.DoFnOperatorFactory<InputT, OutputT>() {
-            @Override
-            public DoFnOperator<InputT, OutputT, RawUnionValue> createDoFnOperator(
-                DoFn<InputT, OutputT> doFn,
-                List<PCollectionView<?>> sideInputs,
-                TupleTag<OutputT> mainOutputTag,
-                List<TupleTag<?>> additionalOutputTags,
-                FlinkStreamingTranslationContext context,
-                WindowingStrategy<?, ?> windowingStrategy,
-                Map<TupleTag<?>, Integer> tagsToLabels,
-                Coder<WindowedValue<InputT>> inputCoder,
-                Coder keyCoder,
-                Map<Integer, PCollectionView<?>> transformedSideInputs) {
-              return new DoFnOperator<>(
-                  doFn,
-                  inputCoder,
-                  mainOutputTag,
-                  additionalOutputTags,
-                  new DoFnOperator.MultiOutputOutputManagerFactory(tagsToLabels),
-                  windowingStrategy,
-                  transformedSideInputs,
-                  sideInputs,
-                  context.getPipelineOptions(),
-                  keyCoder);
-            }
-          });
-    }
-  }
-
-  private static class SplittableProcessElementsStreamingTranslator<
-      InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT, TrackerT>> {
-
-    @Override
-    public void translateNode(
-        SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT, TrackerT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      ParDoTranslationHelper.translateParDo(
-          transform.getName(),
-          transform.newProcessFn(transform.getFn()),
-          (PCollection<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>)
-              context.getInput(transform),
-          transform.getSideInputs(),
-          context.getOutputs(transform),
-          transform.getMainOutputTag(),
-          transform.getAdditionalOutputTags().getAll(),
-          context,
-          new ParDoTranslationHelper.DoFnOperatorFactory<
-              KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT>() {
-            @Override
-            public DoFnOperator<
-                KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>,
-                OutputT,
-                RawUnionValue> createDoFnOperator(
-                    DoFn<
-                        KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>,
-                        OutputT> doFn,
-                    List<PCollectionView<?>> sideInputs,
-                    TupleTag<OutputT> mainOutputTag,
-                    List<TupleTag<?>> additionalOutputTags,
-                    FlinkStreamingTranslationContext context,
-                    WindowingStrategy<?, ?> windowingStrategy,
-                    Map<TupleTag<?>, Integer> tagsToLabels,
-                    Coder<
-                        WindowedValue<
-                            KeyedWorkItem<
-                                String,
-                                ElementAndRestriction<InputT, RestrictionT>>>> inputCoder,
-                    Coder keyCoder,
-                    Map<Integer, PCollectionView<?>> transformedSideInputs) {
-              return new SplittableDoFnOperator<>(
-                  doFn,
-                  inputCoder,
-                  mainOutputTag,
-                  additionalOutputTags,
-                  new DoFnOperator.MultiOutputOutputManagerFactory(tagsToLabels),
-                  windowingStrategy,
-                  transformedSideInputs,
-                  sideInputs,
-                  context.getPipelineOptions(),
-                  keyCoder);
-            }
-          });
-    }
-  }
-
-  private static class CreateViewStreamingTranslator<ElemT, ViewT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      FlinkStreamingViewOverrides.CreateFlinkPCollectionView<ElemT, ViewT>> {
-
-    @Override
-    public void translateNode(
-        FlinkStreamingViewOverrides.CreateFlinkPCollectionView<ElemT, ViewT> transform,
-        FlinkStreamingTranslationContext context) {
-      // just forward
-      DataStream<WindowedValue<List<ElemT>>> inputDataSet =
-          context.getInputDataStream(context.getInput(transform));
-
-      PCollectionView<ViewT> view = context.getOutput(transform);
-
-      context.setOutputDataStream(view, inputDataSet);
-    }
-  }
-
-  private static class WindowAssignTranslator<T>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Assign<T>> {
-
-    @Override
-    public void translateNode(
-        Window.Assign<T> transform,
-        FlinkStreamingTranslationContext context) {
-
-      @SuppressWarnings("unchecked")
-      WindowingStrategy<T, BoundedWindow> windowingStrategy =
-          (WindowingStrategy<T, BoundedWindow>)
-              context.getOutput(transform).getWindowingStrategy();
-
-      TypeInformation<WindowedValue<T>> typeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-      DataStream<WindowedValue<T>> inputDataStream =
-          context.getInputDataStream(context.getInput(transform));
-
-      WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
-
-      FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
-          new FlinkAssignWindows<>(windowFn);
-
-      SingleOutputStreamOperator<WindowedValue<T>> outputDataStream = inputDataStream
-          .flatMap(assignWindowsFunction)
-          .name(context.getOutput(transform).getName())
-          .returns(typeInfo);
-
-      context.setOutputDataStream(context.getOutput(transform), outputDataStream);
-    }
-  }
-
-  private static class ReshuffleTranslatorStreaming<K, InputT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Reshuffle<K, InputT>> {
-
-    @Override
-    public void translateNode(
-        Reshuffle<K, InputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      DataStream<WindowedValue<KV<K, InputT>>> inputDataSet =
-          context.getInputDataStream(context.getInput(transform));
-
-      context.setOutputDataStream(context.getOutput(transform), inputDataSet.rebalance());
-
-    }
-  }
-
-
-  private static class GroupByKeyTranslator<K, InputT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, InputT>> {
-
-    @Override
-    public void translateNode(
-        GroupByKey<K, InputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      PCollection<KV<K, InputT>> input = context.getInput(transform);
-
-      @SuppressWarnings("unchecked")
-      WindowingStrategy<?, BoundedWindow> windowingStrategy =
-          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
-
-      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
-
-      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
-          inputKvCoder.getKeyCoder(),
-          inputKvCoder.getValueCoder(),
-          input.getWindowingStrategy().getWindowFn().windowCoder());
-
-      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
-
-      WindowedValue.
-          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
-          WindowedValue.getFullCoder(
-              workItemCoder,
-              input.getWindowingStrategy().getWindowFn().windowCoder());
-
-      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
-          new CoderTypeInformation<>(windowedWorkItemCoder);
-
-      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
-          inputDataStream
-              .flatMap(new ToKeyedWorkItem<K, InputT>())
-              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
-
-      KeyedStream<
-          WindowedValue<
-              SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
-          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
-
-      SystemReduceFn<K, InputT, Iterable<InputT>, Iterable<InputT>, BoundedWindow> reduceFn =
-          SystemReduceFn.buffering(inputKvCoder.getValueCoder());
-
-      TypeInformation<WindowedValue<KV<K, Iterable<InputT>>>> outputTypeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-      DoFnOperator.DefaultOutputManagerFactory<
-            WindowedValue<KV<K, Iterable<InputT>>>> outputManagerFactory =
-          new DoFnOperator.DefaultOutputManagerFactory<>();
-
-      WindowDoFnOperator<K, InputT, Iterable<InputT>> doFnOperator =
-          new WindowDoFnOperator<>(
-              reduceFn,
-              (Coder) windowedWorkItemCoder,
-              new TupleTag<KV<K, Iterable<InputT>>>("main output"),
-              Collections.<TupleTag<?>>emptyList(),
-              outputManagerFactory,
-              windowingStrategy,
-              new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-              Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-              context.getPipelineOptions(),
-              inputKvCoder.getKeyCoder());
-
-      // our operator excepts WindowedValue<KeyedWorkItem> while our input stream
-      // is WindowedValue<SingletonKeyedWorkItem>, which is fine but Java doesn't like it ...
-      @SuppressWarnings("unchecked")
-      SingleOutputStreamOperator<WindowedValue<KV<K, Iterable<InputT>>>> outDataStream =
-          keyedWorkItemStream
-              .transform(
-                  transform.getName(),
-                  outputTypeInfo,
-                  (OneInputStreamOperator) doFnOperator);
-
-      context.setOutputDataStream(context.getOutput(transform), outDataStream);
-
-    }
-  }
-
-  private static class CombinePerKeyTranslator<K, InputT, OutputT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      Combine.PerKey<K, InputT, OutputT>> {
-
-    @Override
-    boolean canTranslate(
-        Combine.PerKey<K, InputT, OutputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      // if we have a merging window strategy and side inputs we cannot
-      // translate as a proper combine. We have to group and then run the combine
-      // over the final grouped values.
-      PCollection<KV<K, InputT>> input = context.getInput(transform);
-
-      @SuppressWarnings("unchecked")
-      WindowingStrategy<?, BoundedWindow> windowingStrategy =
-          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
-
-      return windowingStrategy.getWindowFn().isNonMerging() || transform.getSideInputs().isEmpty();
-    }
-
-    @Override
-    public void translateNode(
-        Combine.PerKey<K, InputT, OutputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      PCollection<KV<K, InputT>> input = context.getInput(transform);
-
-      @SuppressWarnings("unchecked")
-      WindowingStrategy<?, BoundedWindow> windowingStrategy =
-          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
-
-      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
-
-      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
-          inputKvCoder.getKeyCoder(),
-          inputKvCoder.getValueCoder(),
-          input.getWindowingStrategy().getWindowFn().windowCoder());
-
-      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
-
-      WindowedValue.
-          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
-            WindowedValue.getFullCoder(
-                workItemCoder,
-                input.getWindowingStrategy().getWindowFn().windowCoder());
-
-      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
-          new CoderTypeInformation<>(windowedWorkItemCoder);
-
-      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
-          inputDataStream
-              .flatMap(new ToKeyedWorkItem<K, InputT>())
-              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
-
-      KeyedStream<
-            WindowedValue<
-                SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
-          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
-
-      SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> reduceFn = SystemReduceFn.combining(
-          inputKvCoder.getKeyCoder(),
-          AppliedCombineFn.withInputCoder(
-              transform.getFn(), input.getPipeline().getCoderRegistry(), inputKvCoder));
-
-      TypeInformation<WindowedValue<KV<K, OutputT>>> outputTypeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-      List<PCollectionView<?>> sideInputs = transform.getSideInputs();
-
-      if (sideInputs.isEmpty()) {
-
-        WindowDoFnOperator<K, InputT, OutputT> doFnOperator =
-            new WindowDoFnOperator<>(
-                reduceFn,
-                (Coder) windowedWorkItemCoder,
-                new TupleTag<KV<K, OutputT>>("main output"),
-                Collections.<TupleTag<?>>emptyList(),
-                new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<K, OutputT>>>(),
-                windowingStrategy,
-                new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-                Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-                context.getPipelineOptions(),
-                inputKvCoder.getKeyCoder());
-
-        // our operator excepts WindowedValue<KeyedWorkItem> while our input stream
-        // is WindowedValue<SingletonKeyedWorkItem>, which is fine but Java doesn't like it ...
-        @SuppressWarnings("unchecked")
-        SingleOutputStreamOperator<WindowedValue<KV<K, OutputT>>> outDataStream =
-            keyedWorkItemStream.transform(
-                transform.getName(), outputTypeInfo, (OneInputStreamOperator) doFnOperator);
-
-        context.setOutputDataStream(context.getOutput(transform), outDataStream);
-      } else {
-        Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>> transformSideInputs =
-            transformSideInputs(sideInputs, context);
-
-        WindowDoFnOperator<K, InputT, OutputT> doFnOperator =
-            new WindowDoFnOperator<>(
-                reduceFn,
-                (Coder) windowedWorkItemCoder,
-                new TupleTag<KV<K, OutputT>>("main output"),
-                Collections.<TupleTag<?>>emptyList(),
-                new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<K, OutputT>>>(),
-                windowingStrategy,
-                transformSideInputs.f0,
-                sideInputs,
-                context.getPipelineOptions(),
-                inputKvCoder.getKeyCoder());
-
-        // we have to manually contruct the two-input transform because we're not
-        // allowed to have only one input keyed, normally.
-
-        TwoInputTransformation<
-            WindowedValue<SingletonKeyedWorkItem<K, InputT>>,
-            RawUnionValue,
-            WindowedValue<KV<K, OutputT>>> rawFlinkTransform = new TwoInputTransformation<>(
-            keyedWorkItemStream.getTransformation(),
-            transformSideInputs.f1.broadcast().getTransformation(),
-            transform.getName(),
-            (TwoInputStreamOperator) doFnOperator,
-            outputTypeInfo,
-            keyedWorkItemStream.getParallelism());
-
-        rawFlinkTransform.setStateKeyType(keyedWorkItemStream.getKeyType());
-        rawFlinkTransform.setStateKeySelectors(keyedWorkItemStream.getKeySelector(), null);
-
-        @SuppressWarnings({ "unchecked", "rawtypes" })
-        SingleOutputStreamOperator<WindowedValue<KV<K, OutputT>>> outDataStream =
-            new SingleOutputStreamOperator(
-                keyedWorkItemStream.getExecutionEnvironment(),
-                rawFlinkTransform) {}; // we have to cheat around the ctor being protected
-
-        keyedWorkItemStream.getExecutionEnvironment().addOperator(rawFlinkTransform);
-
-        context.setOutputDataStream(context.getOutput(transform), outDataStream);
-      }
-    }
-  }
-
-  private static class GBKIntoKeyedWorkItemsTranslator<K, InputT>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      SplittableParDo.GBKIntoKeyedWorkItems<K, InputT>> {
-
-    @Override
-    boolean canTranslate(
-        SplittableParDo.GBKIntoKeyedWorkItems<K, InputT> transform,
-        FlinkStreamingTranslationContext context) {
-      return true;
-    }
-
-    @Override
-    public void translateNode(
-        SplittableParDo.GBKIntoKeyedWorkItems<K, InputT> transform,
-        FlinkStreamingTranslationContext context) {
-
-      PCollection<KV<K, InputT>> input = context.getInput(transform);
-
-      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
-
-      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
-          inputKvCoder.getKeyCoder(),
-          inputKvCoder.getValueCoder(),
-          input.getWindowingStrategy().getWindowFn().windowCoder());
-
-
-      WindowedValue.
-          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
-          WindowedValue.getFullCoder(
-              workItemCoder,
-              input.getWindowingStrategy().getWindowFn().windowCoder());
-
-      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
-          new CoderTypeInformation<>(windowedWorkItemCoder);
-
-      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
-
-      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
-          inputDataStream
-              .flatMap(new ToKeyedWorkItem<K, InputT>())
-              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
-
-      KeyedStream<
-          WindowedValue<
-              SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
-          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
-
-      context.setOutputDataStream(context.getOutput(transform), keyedWorkItemStream);
-    }
-  }
-
-  private static class FlattenPCollectionTranslator<T>
-      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
-      Flatten.PCollections<T>> {
-
-    @Override
-    public void translateNode(
-        Flatten.PCollections<T> transform,
-        FlinkStreamingTranslationContext context) {
-      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
-
-      if (allInputs.isEmpty()) {
-
-        // create an empty dummy source to satisfy downstream operations
-        // we cannot create an empty source in Flink, therefore we have to
-        // add the flatMap that simply never forwards the single element
-        DataStreamSource<String> dummySource =
-            context.getExecutionEnvironment().fromElements("dummy");
-
-        DataStream<WindowedValue<T>> result = dummySource.flatMap(
-            new FlatMapFunction<String, WindowedValue<T>>() {
-              @Override
-              public void flatMap(
-                  String s,
-                  Collector<WindowedValue<T>> collector) throws Exception {
-                // never return anything
-              }
-            }).returns(
-            new CoderTypeInformation<>(
-                WindowedValue.getFullCoder(
-                    (Coder<T>) VoidCoder.of(),
-                    GlobalWindow.Coder.INSTANCE)));
-        context.setOutputDataStream(context.getOutput(transform), result);
-
-      } else {
-        DataStream<T> result = null;
-        for (PValue input : allInputs.values()) {
-          DataStream<T> current = context.getInputDataStream(input);
-          result = (result == null) ? current : result.union(current);
-        }
-        context.setOutputDataStream(context.getOutput(transform), result);
-      }
-    }
-  }
-
-  private static class ToKeyedWorkItem<K, InputT>
-      extends RichFlatMapFunction<
-      WindowedValue<KV<K, InputT>>,
-      WindowedValue<SingletonKeyedWorkItem<K, InputT>>> {
-
-    @Override
-    public void flatMap(
-        WindowedValue<KV<K, InputT>> inWithMultipleWindows,
-        Collector<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> out) throws Exception {
-
-      // we need to wrap each one work item per window for now
-      // since otherwise the PushbackSideInputRunner will not correctly
-      // determine whether side inputs are ready
-      //
-      // this is tracked as https://issues.apache.org/jira/browse/BEAM-1850
-      for (WindowedValue<KV<K, InputT>> in : inWithMultipleWindows.explodeWindows()) {
-        SingletonKeyedWorkItem<K, InputT> workItem =
-            new SingletonKeyedWorkItem<>(
-                in.getValue().getKey(),
-                in.withValue(in.getValue().getValue()));
-
-        out.collect(in.withValue(workItem));
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
deleted file mode 100644
index 1a943a3..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Iterables;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.sdk.coders.Coder;
-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.util.WindowedValue;
-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.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-/**
- * Helper for keeping track of which {@link DataStream DataStreams} map
- * to which {@link PTransform PTransforms}.
- */
-class FlinkStreamingTranslationContext {
-
-  private final StreamExecutionEnvironment env;
-  private final PipelineOptions options;
-
-  /**
-   * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
-   * Flink Operator that produced it, after the translation of the correspondinf PTransform
-   * to its Flink equivalent.
-   * */
-  private final Map<PValue, DataStream<?>> dataStreams;
-
-  private AppliedPTransform<?, ?, ?> currentTransform;
-
-  public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
-    this.env = checkNotNull(env);
-    this.options = checkNotNull(options);
-    this.dataStreams = new HashMap<>();
-  }
-
-  public StreamExecutionEnvironment getExecutionEnvironment() {
-    return env;
-  }
-
-  public PipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> DataStream<T> getInputDataStream(PValue value) {
-    return (DataStream<T>) dataStreams.get(value);
-  }
-
-  public void setOutputDataStream(PValue value, DataStream<?> set) {
-    if (!dataStreams.containsKey(value)) {
-      dataStreams.put(value, set);
-    }
-  }
-
-  /**
-   * Sets the AppliedPTransform which carries input/output.
-   * @param currentTransform
-   */
-  public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
-    this.currentTransform = currentTransform;
-  }
-
-  public <T> Coder<WindowedValue<T>> getCoder(PCollection<T> collection) {
-    Coder<T> valueCoder = collection.getCoder();
-
-    return WindowedValue.getFullCoder(
-        valueCoder,
-        collection.getWindowingStrategy().getWindowFn().windowCoder());
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
-    Coder<T> valueCoder = collection.getCoder();
-    WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
-        WindowedValue.getFullCoder(
-            valueCoder,
-            collection.getWindowingStrategy().getWindowFn().windowCoder());
-
-    return new CoderTypeInformation<>(windowedValueCoder);
-  }
-
-
-  @SuppressWarnings("unchecked")
-  public <T extends PValue> T getInput(PTransform<T, ?> transform) {
-    return (T) Iterables.getOnlyElement(currentTransform.getInputs().values());
-  }
-
-  public <T extends PInput> Map<TupleTag<?>, PValue> getInputs(PTransform<T, ?> transform) {
-    return currentTransform.getInputs();
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T extends PValue> T getOutput(PTransform<?, T> transform) {
-    return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values());
-  }
-
-  public <OutputT extends POutput> Map<TupleTag<?>, PValue> getOutputs(
-      PTransform<?, OutputT> transform) {
-    return currentTransform.getOutputs();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
deleted file mode 100644
index f955f2a..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-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.coders.ListCoder;
-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.View;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * Flink streaming overrides for various view (side input) transforms.
- */
-class FlinkStreamingViewOverrides {
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap}
-   * for the Flink runner in streaming mode.
-   */
-  static class StreamingViewAsMap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
-
-    private final transient FlinkRunner runner;
-
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsMap(FlinkRunner runner, View.AsMap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, V>> expand(PCollection<KV<K, V>> input) {
-      PCollectionView<Map<K, V>> view =
-          PCollectionViews.mapView(
-              input,
-              input.getWindowingStrategy(),
-              input.getCoder());
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        inputCoder.getKeyCoder().verifyDeterministic();
-      } catch (Coder.NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-      }
-
-      return input
-          .apply(Combine.globally(new Concatenate<KV<K, V>>()).withoutDefaults())
-          .apply(CreateFlinkPCollectionView.<KV<K, V>, Map<K, V>>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingViewAsMap";
-    }
-  }
-
-  /**
-   * Specialized expansion for {@link
-   * View.AsMultimap View.AsMultimap} for the
-   * Flink runner in streaming mode.
-   */
-  static class StreamingViewAsMultimap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
-
-    private final transient FlinkRunner runner;
-
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, Iterable<V>>> expand(PCollection<KV<K, V>> input) {
-      PCollectionView<Map<K, Iterable<V>>> view =
-          PCollectionViews.multimapView(
-              input,
-              input.getWindowingStrategy(),
-              input.getCoder());
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        inputCoder.getKeyCoder().verifyDeterministic();
-      } catch (Coder.NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-      }
-
-      return input
-          .apply(Combine.globally(new Concatenate<KV<K, V>>()).withoutDefaults())
-          .apply(CreateFlinkPCollectionView.<KV<K, V>, Map<K, Iterable<V>>>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingViewAsMultimap";
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link View.AsList View.AsList} for the
-   * Flink runner in streaming mode.
-   */
-  static class StreamingViewAsList<T>
-      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsList(FlinkRunner runner, View.AsList<T> transform) {}
-
-    @Override
-    public PCollectionView<List<T>> expand(PCollection<T> input) {
-      PCollectionView<List<T>> view =
-          PCollectionViews.listView(
-              input,
-              input.getWindowingStrategy(),
-              input.getCoder());
-
-      return input.apply(Combine.globally(new Concatenate<T>()).withoutDefaults())
-          .apply(CreateFlinkPCollectionView.<T, List<T>>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingViewAsList";
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link View.AsIterable View.AsIterable} for the
-   * Flink runner in streaming mode.
-   */
-  static class StreamingViewAsIterable<T>
-      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable<T> transform) { }
-
-    @Override
-    public PCollectionView<Iterable<T>> expand(PCollection<T> input) {
-      PCollectionView<Iterable<T>> view =
-          PCollectionViews.iterableView(
-              input,
-              input.getWindowingStrategy(),
-              input.getCoder());
-
-      return input.apply(Combine.globally(new Concatenate<T>()).withoutDefaults())
-          .apply(CreateFlinkPCollectionView.<T, Iterable<T>>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingViewAsIterable";
-    }
-  }
-
-  /**
-   * Specialized expansion for
-   * {@link View.AsSingleton View.AsSingleton} for the
-   * Flink runner in streaming mode.
-   */
-  static class StreamingViewAsSingleton<T>
-      extends PTransform<PCollection<T>, PCollectionView<T>> {
-    private View.AsSingleton<T> transform;
-
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PCollectionView<T> expand(PCollection<T> input) {
-      Combine.Globally<T, T> combine = Combine.globally(
-          new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue()));
-      if (!transform.hasDefaultValue()) {
-        combine = combine.withoutDefaults();
-      }
-      return input.apply(combine.asSingletonView());
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingViewAsSingleton";
-    }
-
-    private static class SingletonCombine<T> extends Combine.BinaryCombineFn<T> {
-      private boolean hasDefaultValue;
-      private T defaultValue;
-
-      SingletonCombine(boolean hasDefaultValue, T defaultValue) {
-        this.hasDefaultValue = hasDefaultValue;
-        this.defaultValue = defaultValue;
-      }
-
-      @Override
-      public T apply(T left, T right) {
-        throw new IllegalArgumentException("PCollection with more than one element "
-            + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to "
-            + "combine the PCollection into a single value");
-      }
-
-      @Override
-      public T identity() {
-        if (hasDefaultValue) {
-          return defaultValue;
-        } else {
-          throw new IllegalArgumentException(
-              "Empty PCollection accessed as a singleton view. "
-                  + "Consider setting withDefault to provide a default value");
-        }
-      }
-    }
-  }
-
-  static class StreamingCombineGloballyAsSingletonView<InputT, OutputT>
-      extends PTransform<PCollection<InputT>, PCollectionView<OutputT>> {
-    Combine.GloballyAsSingletonView<InputT, OutputT> transform;
-
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingCombineGloballyAsSingletonView(
-        FlinkRunner runner,
-        Combine.GloballyAsSingletonView<InputT, OutputT> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PCollectionView<OutputT> expand(PCollection<InputT> input) {
-      PCollection<OutputT> combined =
-          input.apply(Combine.globally(transform.getCombineFn())
-              .withoutDefaults()
-              .withFanout(transform.getFanout()));
-
-      PCollectionView<OutputT> view = PCollectionViews.singletonView(
-          combined,
-          combined.getWindowingStrategy(),
-          transform.getInsertDefault(),
-          transform.getInsertDefault()
-              ? transform.getCombineFn().defaultValue() : null,
-          combined.getCoder());
-      return combined
-          .apply(ParDo.of(new WrapAsList<OutputT>()))
-          .apply(CreateFlinkPCollectionView.<OutputT, OutputT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "StreamingCombineGloballyAsSingletonView";
-    }
-  }
-
-  private static class WrapAsList<T> extends DoFn<T, List<T>> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(Collections.singletonList(c.element()));
-    }
-  }
-
-  /**
-   * Combiner that combines {@code T}s into a single {@code List<T>} containing all inputs.
-   *
-   * <p>For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap},
-   * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}.
-   * They require the input {@link PCollection} fits in memory.
-   * For a large {@link PCollection} this is expected to crash!
-   *
-   * @param <T> the type of elements to concatenate.
-   */
-  private static class Concatenate<T> extends Combine.CombineFn<T, List<T>, List<T>> {
-    @Override
-    public List<T> createAccumulator() {
-      return new ArrayList<T>();
-    }
-
-    @Override
-    public List<T> addInput(List<T> accumulator, T input) {
-      accumulator.add(input);
-      return accumulator;
-    }
-
-    @Override
-    public List<T> mergeAccumulators(Iterable<List<T>> accumulators) {
-      List<T> result = createAccumulator();
-      for (List<T> accumulator : accumulators) {
-        result.addAll(accumulator);
-      }
-      return result;
-    }
-
-    @Override
-    public List<T> extractOutput(List<T> accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<List<T>> getAccumulatorCoder(CoderRegistry registry, Coder<T> inputCoder) {
-      return ListCoder.of(inputCoder);
-    }
-
-    @Override
-    public Coder<List<T>> getDefaultOutputCoder(CoderRegistry registry, Coder<T> inputCoder) {
-      return ListCoder.of(inputCoder);
-    }
-  }
-
-  /**
-   * Creates a primitive {@link PCollectionView}.
-   *
-   * <p>For internal use only by runner implementors.
-   *
-   * @param <ElemT> The type of the elements of the input PCollection
-   * @param <ViewT> The type associated with the {@link PCollectionView} used as a side input
-   */
-  public static class CreateFlinkPCollectionView<ElemT, ViewT>
-      extends PTransform<PCollection<List<ElemT>>, PCollectionView<ViewT>> {
-    private PCollectionView<ViewT> view;
-
-    private CreateFlinkPCollectionView(PCollectionView<ViewT> view) {
-      this.view = view;
-    }
-
-    public static <ElemT, ViewT> CreateFlinkPCollectionView<ElemT, ViewT> of(
-        PCollectionView<ViewT> view) {
-      return new CreateFlinkPCollectionView<>(view);
-    }
-
-    @Override
-    public PCollectionView<ViewT> expand(PCollection<List<ElemT>> input) {
-      return view;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
deleted file mode 100644
index 3acc3ea..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline.
- */
-class PipelineTranslationOptimizer extends FlinkPipelineTranslator {
-
-  private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslationOptimizer.class);
-
-  private TranslationMode translationMode;
-
-  private final FlinkPipelineOptions options;
-
-  public PipelineTranslationOptimizer(TranslationMode defaultMode, FlinkPipelineOptions options) {
-    this.translationMode = defaultMode;
-    this.options = options;
-  }
-
-  public TranslationMode getTranslationMode() {
-
-    // override user-specified translation mode
-    if (options.isStreaming()) {
-      return TranslationMode.STREAMING;
-    }
-
-    return translationMode;
-  }
-
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    return CompositeBehavior.ENTER_TRANSFORM;
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformHierarchy.Node node) {}
-
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    Class<? extends PTransform> transformClass = node.getTransform().getClass();
-    if (transformClass == Read.Unbounded.class) {
-      LOG.info("Found {}. Switching to streaming execution.", transformClass);
-      translationMode = TranslationMode.STREAMING;
-    }
-  }
-
-  @Override
-  public void visitValue(PValue value, TransformHierarchy.Node producer) {}
-}


[03/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
new file mode 100644
index 0000000..2bf0bf1
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import com.google.common.collect.Iterators;
+import java.util.Collections;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.coders.Coder;
+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.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MapState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.SetState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+
+/**
+ * {@link StateInternals} that uses a Flink {@link OperatorStateBackend}
+ * to manage the split-distribute state.
+ *
+ * <p>Elements in ListState will be redistributed in round robin fashion
+ * to operators when restarting with a different parallelism.
+ *
+ *  <p>Note:
+ *  Ignore index of key and namespace.
+ *  Just implement BagState.
+ */
+public class FlinkSplitStateInternals<K> implements StateInternals<K> {
+
+  private final OperatorStateBackend stateBackend;
+
+  public FlinkSplitStateInternals(OperatorStateBackend stateBackend) {
+    this.stateBackend = stateBackend;
+  }
+
+  @Override
+  public K getKey() {
+    return null;
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address) {
+
+    return state(namespace, address, StateContexts.nullContext());
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address,
+      final StateContext<?> context) {
+
+    return address.bind(new StateTag.StateBinder<K>() {
+
+      @Override
+      public <T> ValueState<T> bindValue(
+          StateTag<? super K, ValueState<T>> address,
+          Coder<T> coder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", ValueState.class.getSimpleName()));
+      }
+
+      @Override
+      public <T> BagState<T> bindBag(
+          StateTag<? super K, BagState<T>> address,
+          Coder<T> elemCoder) {
+
+        return new FlinkSplitBagState<>(stateBackend, address, namespace, elemCoder);
+      }
+
+      @Override
+      public <T> SetState<T> bindSet(
+          StateTag<? super K, SetState<T>> address,
+          Coder<T> elemCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", SetState.class.getSimpleName()));
+      }
+
+      @Override
+      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
+          StateTag<? super K, MapState<KeyT, ValueT>> spec,
+          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", MapState.class.getSimpleName()));
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT>
+      bindCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException("bindCombiningValue is not supported.");
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported.");
+
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          CombineWithContext.KeyedCombineFnWithContext<
+              ? super K, InputT, AccumT, OutputT> combineFn) {
+        throw new UnsupportedOperationException(
+            "bindKeyedCombiningValueWithContext is not supported.");
+      }
+
+      @Override
+      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
+          StateTag<? super K, WatermarkHoldState<W>> address,
+          OutputTimeFn<? super W> outputTimeFn) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", CombiningState.class.getSimpleName()));
+      }
+    });
+  }
+
+  private static class FlinkSplitBagState<K, T> implements BagState<T> {
+
+    private final ListStateDescriptor<T> descriptor;
+    private OperatorStateBackend flinkStateBackend;
+    private final StateNamespace namespace;
+    private final StateTag<? super K, BagState<T>> address;
+
+    FlinkSplitBagState(
+        OperatorStateBackend flinkStateBackend,
+        StateTag<? super K, BagState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+      this.flinkStateBackend = flinkStateBackend;
+      this.namespace = namespace;
+      this.address = address;
+
+      CoderTypeInformation<T> typeInfo =
+          new CoderTypeInformation<>(coder);
+
+      descriptor = new ListStateDescriptor<>(address.getId(),
+          typeInfo.createSerializer(new ExecutionConfig()));
+    }
+
+    @Override
+    public void add(T input) {
+      try {
+        flinkStateBackend.getOperatorState(descriptor).add(input);
+      } catch (Exception e) {
+        throw new RuntimeException("Error updating state.", e);
+      }
+    }
+
+    @Override
+    public BagState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public Iterable<T> read() {
+      try {
+        Iterable<T> result = flinkStateBackend.getOperatorState(descriptor).get();
+        return result != null ? result : Collections.<T>emptyList();
+      } catch (Exception e) {
+        throw new RuntimeException("Error updating state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            Iterable<T> result = flinkStateBackend.getOperatorState(descriptor).get();
+            // PartitionableListState.get() return empty collection When there is no element,
+            // KeyedListState different. (return null)
+            return result == null || Iterators.size(result.iterator()) == 0;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getOperatorState(descriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkSplitBagState<?, ?> that = (FlinkSplitBagState<?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
new file mode 100644
index 0000000..4f961e5
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -0,0 +1,1053 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import com.google.common.collect.Lists;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+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.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.CoderUtils;
+import org.apache.beam.sdk.util.CombineContextFactory;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MapState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.SetState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.joda.time.Instant;
+
+/**
+ * {@link StateInternals} that uses a Flink {@link KeyedStateBackend} to manage state.
+ *
+ * <p>Note: In the Flink streaming runner the key is always encoded
+ * using an {@link Coder} and stored in a {@link ByteBuffer}.
+ */
+public class FlinkStateInternals<K> implements StateInternals<K> {
+
+  private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+  private Coder<K> keyCoder;
+
+  // on recovery, these will no be properly set because we don't
+  // know which watermark hold states there are in the Flink State Backend
+  private final Map<String, Instant> watermarkHolds = new HashMap<>();
+
+  public FlinkStateInternals(KeyedStateBackend<ByteBuffer> flinkStateBackend, Coder<K> keyCoder) {
+    this.flinkStateBackend = flinkStateBackend;
+    this.keyCoder = keyCoder;
+  }
+
+  /**
+   * Returns the minimum over all watermark holds.
+   */
+  public Instant watermarkHold() {
+    long min = Long.MAX_VALUE;
+    for (Instant hold: watermarkHolds.values()) {
+      min = Math.min(min, hold.getMillis());
+    }
+    return new Instant(min);
+  }
+
+  @Override
+  public K getKey() {
+    ByteBuffer keyBytes = flinkStateBackend.getCurrentKey();
+    try {
+      return CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array());
+    } catch (CoderException e) {
+      throw new RuntimeException("Error decoding key.", e);
+    }
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address) {
+
+    return state(namespace, address, StateContexts.nullContext());
+  }
+
+  @Override
+  public <T extends State> T state(
+      final StateNamespace namespace,
+      StateTag<? super K, T> address,
+      final StateContext<?> context) {
+
+    return address.bind(new StateTag.StateBinder<K>() {
+
+      @Override
+      public <T> ValueState<T> bindValue(
+          StateTag<? super K, ValueState<T>> address,
+          Coder<T> coder) {
+
+        return new FlinkValueState<>(flinkStateBackend, address, namespace, coder);
+      }
+
+      @Override
+      public <T> BagState<T> bindBag(
+          StateTag<? super K, BagState<T>> address,
+          Coder<T> elemCoder) {
+
+        return new FlinkBagState<>(flinkStateBackend, address, namespace, elemCoder);
+      }
+
+      @Override
+      public <T> SetState<T> bindSet(
+          StateTag<? super K, SetState<T>> address,
+          Coder<T> elemCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", SetState.class.getSimpleName()));
+      }
+
+      @Override
+      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
+          StateTag<? super K, MapState<KeyT, ValueT>> spec,
+          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", MapState.class.getSimpleName()));
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT>
+      bindCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+
+        return new FlinkCombiningState<>(
+            flinkStateBackend, address, combineFn, namespace, accumCoder);
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+        return new FlinkKeyedCombiningState<>(
+            flinkStateBackend,
+            address,
+            combineFn,
+            namespace,
+            accumCoder,
+            FlinkStateInternals.this);
+      }
+
+      @Override
+      public <InputT, AccumT, OutputT>
+      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+          Coder<AccumT> accumCoder,
+          CombineWithContext.KeyedCombineFnWithContext<
+              ? super K, InputT, AccumT, OutputT> combineFn) {
+        return new FlinkCombiningStateWithContext<>(
+            flinkStateBackend,
+            address,
+            combineFn,
+            namespace,
+            accumCoder,
+            FlinkStateInternals.this,
+            CombineContextFactory.createFromStateContext(context));
+      }
+
+      @Override
+      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
+          StateTag<? super K, WatermarkHoldState<W>> address,
+          OutputTimeFn<? super W> outputTimeFn) {
+
+        return new FlinkWatermarkHoldState<>(
+            flinkStateBackend, FlinkStateInternals.this, address, namespace, outputTimeFn);
+      }
+    });
+  }
+
+  private static class FlinkValueState<K, T> implements ValueState<T> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, ValueState<T>> address;
+    private final ValueStateDescriptor<T> flinkStateDescriptor;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+
+    FlinkValueState(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        StateTag<? super K, ValueState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+
+      this.namespace = namespace;
+      this.address = address;
+      this.flinkStateBackend = flinkStateBackend;
+
+      CoderTypeInformation<T> typeInfo = new CoderTypeInformation<>(coder);
+
+      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
+    }
+
+    @Override
+    public void write(T input) {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).update(input);
+      } catch (Exception e) {
+        throw new RuntimeException("Error updating state.", e);
+      }
+    }
+
+    @Override
+    public ValueState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public T read() {
+      try {
+        return flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).value();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkValueState<?, ?> that = (FlinkValueState<?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private static class FlinkBagState<K, T> implements BagState<T> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, BagState<T>> address;
+    private final ListStateDescriptor<T> flinkStateDescriptor;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+
+    FlinkBagState(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        StateTag<? super K, BagState<T>> address,
+        StateNamespace namespace,
+        Coder<T> coder) {
+
+      this.namespace = namespace;
+      this.address = address;
+      this.flinkStateBackend = flinkStateBackend;
+
+      CoderTypeInformation<T> typeInfo = new CoderTypeInformation<>(coder);
+
+      flinkStateDescriptor = new ListStateDescriptor<>(address.getId(), typeInfo);
+    }
+
+    @Override
+    public void add(T input) {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).add(input);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to bag state.", e);
+      }
+    }
+
+    @Override
+    public BagState<T> readLater() {
+      return this;
+    }
+
+    @Override
+    public Iterable<T> read() {
+      try {
+        Iterable<T> result = flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).get();
+
+        return result != null ? result : Collections.<T>emptyList();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            Iterable<T> result = flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor).get();
+            return result == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkBagState<?, ?> that = (FlinkBagState<?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private static class FlinkCombiningState<K, InputT, AccumT, OutputT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
+    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+
+    FlinkCombiningState(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder) {
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+      this.flinkStateBackend = flinkStateBackend;
+
+      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
+
+      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          current = combineFn.createAccumulator();
+        }
+        current = combineFn.addInput(current, value);
+        state.update(current);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state." , e);
+      }
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+              namespace.stringKey(),
+              StringSerializer.INSTANCE,
+              flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          state.update(accum);
+        } else {
+          current = combineFn.mergeAccumulators(Lists.newArrayList(current, accum));
+          state.update(current);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state.", e);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      try {
+        return flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).value();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(accumulators);
+    }
+
+    @Override
+    public OutputT read() {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT accum = state.value();
+        if (accum != null) {
+          return combineFn.extractOutput(accum);
+        } else {
+          return combineFn.extractOutput(combineFn.createAccumulator());
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor).value() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkCombiningState<?, ?, ?, ?> that =
+          (FlinkCombiningState<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private static class FlinkKeyedCombiningState<K, InputT, AccumT, OutputT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn;
+    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+    private final FlinkStateInternals<K> flinkStateInternals;
+
+    FlinkKeyedCombiningState(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder,
+        FlinkStateInternals<K> flinkStateInternals) {
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+      this.flinkStateBackend = flinkStateBackend;
+      this.flinkStateInternals = flinkStateInternals;
+
+      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
+
+      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          current = combineFn.createAccumulator(flinkStateInternals.getKey());
+        }
+        current = combineFn.addInput(flinkStateInternals.getKey(), current, value);
+        state.update(current);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state." , e);
+      }
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          state.update(accum);
+        } else {
+          current = combineFn.mergeAccumulators(
+              flinkStateInternals.getKey(),
+              Lists.newArrayList(current, accum));
+          state.update(current);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state.", e);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      try {
+        return flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).value();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators);
+    }
+
+    @Override
+    public OutputT read() {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT accum = state.value();
+        if (accum != null) {
+          return combineFn.extractOutput(flinkStateInternals.getKey(), accum);
+        } else {
+          return combineFn.extractOutput(
+              flinkStateInternals.getKey(),
+              combineFn.createAccumulator(flinkStateInternals.getKey()));
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor).value() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkKeyedCombiningState<?, ?, ?, ?> that =
+          (FlinkKeyedCombiningState<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private static class FlinkCombiningStateWithContext<K, InputT, AccumT, OutputT>
+      implements CombiningState<InputT, AccumT, OutputT> {
+
+    private final StateNamespace namespace;
+    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
+    private final CombineWithContext.KeyedCombineFnWithContext<
+        ? super K, InputT, AccumT, OutputT> combineFn;
+    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+    private final FlinkStateInternals<K> flinkStateInternals;
+    private final CombineWithContext.Context context;
+
+    FlinkCombiningStateWithContext(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
+        CombineWithContext.KeyedCombineFnWithContext<
+            ? super K, InputT, AccumT, OutputT> combineFn,
+        StateNamespace namespace,
+        Coder<AccumT> accumCoder,
+        FlinkStateInternals<K> flinkStateInternals,
+        CombineWithContext.Context context) {
+
+      this.namespace = namespace;
+      this.address = address;
+      this.combineFn = combineFn;
+      this.flinkStateBackend = flinkStateBackend;
+      this.flinkStateInternals = flinkStateInternals;
+      this.context = context;
+
+      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
+
+      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
+    }
+
+    @Override
+    public CombiningState<InputT, AccumT, OutputT> readLater() {
+      return this;
+    }
+
+    @Override
+    public void add(InputT value) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          current = combineFn.createAccumulator(flinkStateInternals.getKey(), context);
+        }
+        current = combineFn.addInput(flinkStateInternals.getKey(), current, value, context);
+        state.update(current);
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state." , e);
+      }
+    }
+
+    @Override
+    public void addAccum(AccumT accum) {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT current = state.value();
+        if (current == null) {
+          state.update(accum);
+        } else {
+          current = combineFn.mergeAccumulators(
+              flinkStateInternals.getKey(),
+              Lists.newArrayList(current, accum),
+              context);
+          state.update(current);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error adding to state.", e);
+      }
+    }
+
+    @Override
+    public AccumT getAccum() {
+      try {
+        return flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).value();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators, context);
+    }
+
+    @Override
+    public OutputT read() {
+      try {
+        org.apache.flink.api.common.state.ValueState<AccumT> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+
+        AccumT accum = state.value();
+        return combineFn.extractOutput(flinkStateInternals.getKey(), accum, context);
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor).value() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+    }
+
+    @Override
+    public void clear() {
+      try {
+        flinkStateBackend.getPartitionedState(
+            namespace.stringKey(),
+            StringSerializer.INSTANCE,
+            flinkStateDescriptor).clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error clearing state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkCombiningStateWithContext<?, ?, ?, ?> that =
+          (FlinkCombiningStateWithContext<?, ?, ?, ?>) o;
+
+      return namespace.equals(that.namespace) && address.equals(that.address);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = namespace.hashCode();
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+  }
+
+  private static class FlinkWatermarkHoldState<K, W extends BoundedWindow>
+      implements WatermarkHoldState<W> {
+    private final StateTag<? super K, WatermarkHoldState<W>> address;
+    private final OutputTimeFn<? super W> outputTimeFn;
+    private final StateNamespace namespace;
+    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+    private final FlinkStateInternals<K> flinkStateInternals;
+    private final ValueStateDescriptor<Instant> flinkStateDescriptor;
+
+    public FlinkWatermarkHoldState(
+        KeyedStateBackend<ByteBuffer> flinkStateBackend,
+        FlinkStateInternals<K> flinkStateInternals,
+        StateTag<? super K, WatermarkHoldState<W>> address,
+        StateNamespace namespace,
+        OutputTimeFn<? super W> outputTimeFn) {
+      this.address = address;
+      this.outputTimeFn = outputTimeFn;
+      this.namespace = namespace;
+      this.flinkStateBackend = flinkStateBackend;
+      this.flinkStateInternals = flinkStateInternals;
+
+      CoderTypeInformation<Instant> typeInfo = new CoderTypeInformation<>(InstantCoder.of());
+      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
+    }
+
+    @Override
+    public OutputTimeFn<? super W> getOutputTimeFn() {
+      return outputTimeFn;
+    }
+
+    @Override
+    public WatermarkHoldState<W> readLater() {
+      return this;
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty() {
+      return new ReadableState<Boolean>() {
+        @Override
+        public Boolean read() {
+          try {
+            return flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor).value() == null;
+          } catch (Exception e) {
+            throw new RuntimeException("Error reading state.", e);
+          }
+        }
+
+        @Override
+        public ReadableState<Boolean> readLater() {
+          return this;
+        }
+      };
+
+    }
+
+    @Override
+    public void add(Instant value) {
+      try {
+        org.apache.flink.api.common.state.ValueState<Instant> state =
+            flinkStateBackend.getPartitionedState(
+              namespace.stringKey(),
+              StringSerializer.INSTANCE,
+              flinkStateDescriptor);
+
+        Instant current = state.value();
+        if (current == null) {
+          state.update(value);
+          flinkStateInternals.watermarkHolds.put(namespace.stringKey(), value);
+        } else {
+          Instant combined = outputTimeFn.combine(current, value);
+          state.update(combined);
+          flinkStateInternals.watermarkHolds.put(namespace.stringKey(), combined);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error updating state.", e);
+      }
+    }
+
+    @Override
+    public Instant read() {
+      try {
+        org.apache.flink.api.common.state.ValueState<Instant> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+        return state.value();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public void clear() {
+      flinkStateInternals.watermarkHolds.remove(namespace.stringKey());
+      try {
+        org.apache.flink.api.common.state.ValueState<Instant> state =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(),
+                StringSerializer.INSTANCE,
+                flinkStateDescriptor);
+        state.clear();
+      } catch (Exception e) {
+        throw new RuntimeException("Error reading state.", e);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      FlinkWatermarkHoldState<?, ?> that = (FlinkWatermarkHoldState<?, ?>) o;
+
+      if (!address.equals(that.address)) {
+        return false;
+      }
+      if (!outputTimeFn.equals(that.outputTimeFn)) {
+        return false;
+      }
+      return namespace.equals(that.namespace);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = address.hashCode();
+      result = 31 * result + outputTimeFn.hashCode();
+      result = 31 * result + namespace.hashCode();
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java
new file mode 100644
index 0000000..b38a520
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.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 org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import java.io.DataOutputStream;
+
+/**
+ * This interface is used to checkpoint key-groups state.
+ */
+public interface KeyGroupCheckpointedOperator extends KeyGroupRestoringOperator{
+  /**
+   * Snapshots the state for a given {@code keyGroupIdx}.
+   *
+   * <p>AbstractStreamOperator would call this hook in
+   * AbstractStreamOperator.snapshotState() while iterating over the key groups.
+   * @param keyGroupIndex the id of the key-group to be put in the snapshot.
+   * @param out the stream to write to.
+   */
+  void snapshotKeyGroupState(int keyGroupIndex, DataOutputStream out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java
new file mode 100644
index 0000000..2bdfc6e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
+
+import java.io.DataInputStream;
+
+/**
+ * This interface is used to restore key-groups state.
+ */
+public interface KeyGroupRestoringOperator {
+  /**
+   * Restore the state for a given {@code keyGroupIndex}.
+   * @param keyGroupIndex the id of the key-group to be put in the snapshot.
+   * @param in the stream to read from.
+   */
+  void restoreKeyGroupState(int keyGroupIndex, DataInputStream in) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
new file mode 100644
index 0000000..0004e9e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal state implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.state;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/resources/log4j.properties b/runners/flink/src/main/resources/log4j.properties
new file mode 100644
index 0000000..4b6a708
--- /dev/null
+++ b/runners/flink/src/main/resources/log4j.properties
@@ -0,0 +1,23 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=OFF,console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
new file mode 100644
index 0000000..10d6d9d
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import org.apache.beam.runners.flink.translation.types.EncodedValueComparator;
+import org.apache.beam.runners.flink.translation.types.EncodedValueTypeInformation;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.junit.Assert;
+
+/**
+ * Test for {@link EncodedValueComparator}.
+ */
+public class EncodedValueComparatorTest extends ComparatorTestBase<byte[]> {
+
+  @Override
+  protected TypeComparator<byte[]> createComparator(boolean ascending) {
+    return new EncodedValueTypeInformation().createComparator(ascending, new ExecutionConfig());
+  }
+
+  @Override
+  protected TypeSerializer<byte[]> createSerializer() {
+    return new EncodedValueTypeInformation().createSerializer(new ExecutionConfig());
+  }
+
+  @Override
+  protected void deepEquals(String message, byte[] should, byte[] is) {
+    Assert.assertArrayEquals(message, should, is);
+  }
+
+  @Override
+  protected byte[][] getSortedTestData() {
+    StringUtf8Coder coder = StringUtf8Coder.of();
+
+    try {
+      return new byte[][]{
+          CoderUtils.encodeToByteArray(coder, ""),
+          CoderUtils.encodeToByteArray(coder, "Lorem Ipsum Dolor Omit Longer"),
+          CoderUtils.encodeToByteArray(coder, "aaaa"),
+          CoderUtils.encodeToByteArray(coder, "abcd"),
+          CoderUtils.encodeToByteArray(coder, "abce"),
+          CoderUtils.encodeToByteArray(coder, "abdd"),
+          CoderUtils.encodeToByteArray(coder, "accd"),
+          CoderUtils.encodeToByteArray(coder, "bbcd")
+      };
+    } catch (CoderException e) {
+      throw new RuntimeException("Could not encode values.", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
new file mode 100644
index 0000000..d9d174c
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.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 org.apache.beam.runners.flink;
+
+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.
+ */
+public class FlinkRunnerRegistrarTest {
+
+  @Test
+  public void testFullName() {
+    String[] args =
+        new String[] {String.format("--runner=%s", FlinkRunner.class.getName())};
+    PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
+    assertEquals(opts.getRunner(), FlinkRunner.class);
+  }
+
+  @Test
+  public void testClassName() {
+    String[] args =
+        new String[] {String.format("--runner=%s", FlinkRunner.class.getSimpleName())};
+    PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
+    assertEquals(opts.getRunner(), FlinkRunner.class);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
new file mode 100644
index 0000000..d6240c4
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+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 org.apache.beam.sdk.Pipeline} for testing Dataflow programs on the
+ * {@link FlinkRunner}.
+ */
+public class FlinkTestPipeline extends Pipeline {
+
+  /**
+   * Creates and returns a new test pipeline for batch execution.
+   *
+   * <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() {
+    return create(false);
+  }
+
+  /**
+   * Creates and returns a new test pipeline for streaming execution.
+   *
+   * <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
+   */
+  public static FlinkTestPipeline createForStreaming() {
+    return create(true);
+  }
+
+  /**
+   * Creates and returns a new test pipeline for streaming or batch execution.
+   *
+   * <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.
+   * @return The Test Pipeline.
+   */
+  private static FlinkTestPipeline create(boolean streaming) {
+    TestFlinkRunner flinkRunner = TestFlinkRunner.create(streaming);
+    return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
+  }
+
+  private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
+              PipelineOptions options) {
+    super(runner, options);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
new file mode 100644
index 0000000..06187f6
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.DoFn;
+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.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.joda.time.Instant;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests for serialization and deserialization of {@link PipelineOptions} in {@link DoFnOperator}.
+ */
+public class PipelineOptionsTest {
+
+  /**
+   * Pipeline options.
+   */
+  public interface MyOptions extends FlinkPipelineOptions {
+    @Description("Bla bla bla")
+    @Default.String("Hello")
+    String getTestOption();
+    void setTestOption(String value);
+  }
+
+  private static MyOptions options;
+  private static SerializedPipelineOptions serializedOptions;
+
+  private static final String[] args = new String[]{"--testOption=nothing"};
+
+  @BeforeClass
+  public static void beforeTest() {
+    options = PipelineOptionsFactory.fromArgs(args).as(MyOptions.class);
+    serializedOptions = new SerializedPipelineOptions(options);
+  }
+
+  @Test
+  public void testDeserialization() {
+    MyOptions deserializedOptions = serializedOptions.getPipelineOptions().as(MyOptions.class);
+    assertEquals("nothing", deserializedOptions.getTestOption());
+  }
+
+  @Test
+  public void testIgnoredFieldSerialization() {
+    FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+    options.setStateBackend(new MemoryStateBackend());
+
+    FlinkPipelineOptions deserialized =
+        new SerializedPipelineOptions(options).getPipelineOptions().as(FlinkPipelineOptions.class);
+
+    assertNull(deserialized.getStateBackend());
+  }
+
+  @Test
+  public void testCaching() {
+    PipelineOptions deserializedOptions =
+        serializedOptions.getPipelineOptions().as(PipelineOptions.class);
+
+    assertNotNull(deserializedOptions);
+    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
+    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
+    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
+  }
+
+  @Test(expected = Exception.class)
+  public void testNonNull() {
+    new SerializedPipelineOptions(null);
+  }
+
+  @Test(expected = Exception.class)
+  public void parDoBaseClassPipelineOptionsNullTest() {
+    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
+        new TestDoFn(),
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()),
+        new TupleTag<String>("main-output"),
+        Collections.<TupleTag<?>>emptyList(),
+        new DoFnOperator.DefaultOutputManagerFactory<String>(),
+        WindowingStrategy.globalDefault(),
+        new HashMap<Integer, PCollectionView<?>>(),
+        Collections.<PCollectionView<?>>emptyList(),
+        null,
+        null);
+
+  }
+
+  /**
+   * Tests that PipelineOptions are present after serialization.
+   */
+  @Test
+  public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception {
+
+    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
+        new TestDoFn(),
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()),
+        new TupleTag<String>("main-output"),
+        Collections.<TupleTag<?>>emptyList(),
+        new DoFnOperator.DefaultOutputManagerFactory<String>(),
+        WindowingStrategy.globalDefault(),
+        new HashMap<Integer, PCollectionView<?>>(),
+        Collections.<PCollectionView<?>>emptyList(),
+        options,
+        null);
+
+    final byte[] serialized = SerializationUtils.serialize(doFnOperator);
+
+    @SuppressWarnings("unchecked")
+    DoFnOperator<Object, Object, Object> deserialized =
+        (DoFnOperator<Object, Object, Object>) SerializationUtils.deserialize(serialized);
+
+    TypeInformation<WindowedValue<Object>> typeInformation = TypeInformation.of(
+        new TypeHint<WindowedValue<Object>>() {});
+
+    OneInputStreamOperatorTestHarness<WindowedValue<Object>, Object> testHarness =
+        new OneInputStreamOperatorTestHarness<>(deserialized,
+            typeInformation.createSerializer(new ExecutionConfig()));
+
+    testHarness.open();
+
+    // execute once to access options
+    testHarness.processElement(new StreamRecord<>(
+        WindowedValue.of(
+            new Object(),
+            Instant.now(),
+            GlobalWindow.INSTANCE,
+            PaneInfo.NO_FIRING)));
+
+    testHarness.close();
+
+  }
+
+
+  private static class TestDoFn extends DoFn<String, String> {
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      Assert.assertNotNull(c.getPipelineOptions());
+      Assert.assertEquals(
+          options.getTestOption(),
+          c.getPipelineOptions().as(MyOptions.class).getTestOption());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
new file mode 100644
index 0000000..44c9017
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import com.google.common.base.Joiner;
+import java.io.File;
+import java.net.URI;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Reads from a bounded source in batch execution.
+ */
+public class ReadSourceITCase extends JavaProgramTestBase {
+
+  protected String resultPath;
+
+  public ReadSourceITCase(){
+  }
+
+  private static final String[] EXPECTED_RESULT = new String[] {
+     "0", "1", "2", "3", "4", "5", "6", "7", "8", "9"};
+
+  @Override
+  protected void preSubmit() throws Exception {
+    resultPath = getTempDirPath("result");
+
+    // need to create the dir, otherwise Beam sinks don't
+    // work for these tests
+
+    if (!new File(new URI(resultPath)).mkdirs()) {
+      throw new RuntimeException("Could not create output dir.");
+    }
+  }
+
+  @Override
+  protected void postSubmit() throws Exception {
+    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+  }
+
+  @Override
+  protected void testProgram() throws Exception {
+    runProgram(resultPath);
+  }
+
+  private static void runProgram(String resultPath) throws Exception {
+
+    Pipeline p = FlinkTestPipeline.createForBatch();
+
+    PCollection<String> result = p
+        .apply(CountingInput.upTo(10))
+        .apply(ParDo.of(new DoFn<Long, String>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) throws Exception {
+            c.output(c.element().toString());
+          }
+        }));
+
+    result.apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part"));
+
+    p.run();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java
new file mode 100644
index 0000000..79b7882
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import com.google.common.base.Joiner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+/**
+ * Reads from a bounded source in streaming.
+ */
+public class ReadSourceStreamingITCase extends StreamingProgramTestBase {
+
+  protected String resultPath;
+
+  public ReadSourceStreamingITCase(){
+  }
+
+  private static final String[] EXPECTED_RESULT = new String[] {
+     "0", "1", "2", "3", "4", "5", "6", "7", "8", "9"};
+
+  @Override
+  protected void preSubmit() throws Exception {
+    resultPath = getTempDirPath("result");
+  }
+
+  @Override
+  protected void postSubmit() throws Exception {
+    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+  }
+
+  @Override
+  protected void testProgram() throws Exception {
+    runProgram(resultPath);
+  }
+
+  private static void runProgram(String resultPath) {
+
+    Pipeline p = FlinkTestPipeline.createForStreaming();
+
+    p
+      .apply(CountingInput.upTo(10))
+      .apply(ParDo.of(new DoFn<Long, String>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) throws Exception {
+            c.output(c.element().toString());
+          }
+        }))
+      .apply(TextIO.Write.to(resultPath));
+
+    p.run();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
new file mode 100644
index 0000000..38b790e
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.flink;
+
+import static org.junit.Assert.assertNotNull;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URI;
+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.io.Write;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Tests the translation of custom Write sinks.
+ */
+public class WriteSinkITCase extends JavaProgramTestBase {
+
+  protected String resultPath;
+
+  public WriteSinkITCase(){
+  }
+
+  static final String[] EXPECTED_RESULT = new String[] {
+      "Joe red 3", "Mary blue 4", "Max yellow 23"};
+
+  @Override
+  protected void preSubmit() throws Exception {
+    resultPath = getTempDirPath("result-" + System.nanoTime());
+  }
+
+  @Override
+  protected void postSubmit() throws Exception {
+    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+  }
+
+  @Override
+  protected void testProgram() throws Exception {
+    runProgram(resultPath);
+  }
+
+  @Override
+  public void stopCluster() throws Exception {
+    try {
+      super.stopCluster();
+    } catch (final IOException ioe) {
+      if (ioe.getMessage().startsWith("Unable to delete file")) {
+        // that's ok for the test itself, just the OS playing with us on cleanup phase
+      }
+    }
+  }
+
+  private static void runProgram(String resultPath) {
+    Pipeline p = FlinkTestPipeline.createForBatch();
+
+    p.apply(Create.of(ImmutableList.copyOf(EXPECTED_RESULT))).setCoder(StringUtf8Coder.of())
+      .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
+
+    p.run();
+  }
+
+  /**
+   * Simple custom sink which writes to a file.
+   */
+  private static class MyCustomSink extends Sink<String> {
+
+    private final String resultPath;
+
+    public MyCustomSink(String resultPath) {
+      this.resultPath = resultPath;
+    }
+
+    @Override
+    public void validate(PipelineOptions options) {
+      assertNotNull(options);
+    }
+
+    @Override
+    public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
+      return new MyWriteOperation();
+    }
+
+    private class MyWriteOperation extends WriteOperation<String, String> {
+
+      @Override
+      public Coder<String> getWriterResultCoder() {
+        return StringUtf8Coder.of();
+      }
+
+      @Override
+      public void initialize(PipelineOptions options) throws Exception {
+
+      }
+
+      @Override
+      public void setWindowedWrites(boolean windowedWrites) {
+
+      }
+
+      @Override
+      public void finalize(Iterable<String> writerResults, PipelineOptions options)
+          throws Exception {
+
+      }
+
+      @Override
+      public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
+        return new MyWriter();
+      }
+
+      @Override
+      public Sink<String> getSink() {
+        return MyCustomSink.this;
+      }
+
+      /**
+       * Simple Writer which writes to a file.
+       */
+      private class MyWriter extends Writer<String, String> {
+
+        private PrintWriter internalWriter;
+
+        @Override
+        public final void openWindowed(String uId,
+                                       BoundedWindow window,
+                                       PaneInfo paneInfo,
+                                       int shard,
+                                       int numShards) throws Exception {
+          throw new UnsupportedOperationException("Windowed writes not supported.");
+        }
+
+        @Override
+        public final void openUnwindowed(String uId, int shard, int numShards) throws Exception {
+          Path path = new Path(resultPath + "/" + uId);
+          FileSystem.get(new URI("file:///")).create(path, false);
+          internalWriter = new PrintWriter(new File(path.toUri()));
+        }
+
+        @Override
+        public void cleanup() throws Exception {
+
+        }
+
+        @Override
+        public void write(String value) throws Exception {
+          internalWriter.println(value);
+        }
+
+        @Override
+        public String close() throws Exception {
+          internalWriter.close();
+          return resultPath;
+        }
+
+        @Override
+        public WriteOperation<String, String> getWriteOperation() {
+          return MyWriteOperation.this;
+        }
+      }
+    }
+  }
+
+}
+


[16/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
deleted file mode 100644
index 18343ef..0000000
--- a/runners/flink/runner/pom.xml
+++ /dev/null
@@ -1,330 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-    Licensed to the Apache Software Foundation (ASF) under one or more
-    contributor license agreements.  See the NOTICE file distributed with
-    this work for additional information regarding copyright ownership.
-    The ASF licenses this file to You under the Apache License, Version 2.0
-    (the "License"); you may not use this file except in compliance with
-    the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software
-    distributed under the License is distributed on an "AS IS" BASIS,
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and
-    limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-runners-flink_2.10</artifactId>
-
-  <name>Apache Beam :: Runners :: Flink :: Core</name>
-
-  <packaging>jar</packaging>
-
-  <profiles>
-    <profile>
-      <id>local-validates-runner-tests</id>
-      <activation><activeByDefault>false</activeByDefault></activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <executions>
-
-              <!-- This configures the inherited validates-runner-tests
-                   execution to execute with a local Flink instance. -->
-              <execution>
-                <id>validates-runner-tests</id>
-                <phase>integration-test</phase>
-                <goals>
-                  <goal>test</goal>
-                </goals>
-                <configuration>
-                  <groups>org.apache.beam.sdk.testing.ValidatesRunner</groups>
-                  <excludedGroups>
-                    org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders,
-                    org.apache.beam.sdk.testing.UsesSplittableParDo,
-                    org.apache.beam.sdk.testing.UsesAttemptedMetrics,
-                    org.apache.beam.sdk.testing.UsesCommittedMetrics,
-                    org.apache.beam.sdk.testing.UsesTestStream
-                  </excludedGroups>
-                  <parallel>none</parallel>
-                  <failIfNoTests>true</failIfNoTests>
-                  <dependenciesToScan>
-                    <dependency>org.apache.beam:beam-sdks-java-core</dependency>
-                  </dependenciesToScan>
-                  <systemPropertyVariables>
-                    <beamTestPipelineOptions>
-                      [
-                      "--runner=TestFlinkRunner",
-                      "--streaming=false"
-                      ]
-                    </beamTestPipelineOptions>
-                  </systemPropertyVariables>
-                </configuration>
-              </execution>
-
-              <!-- This second execution runs the tests in streaming mode -->
-              <execution>
-                <id>streaming-validates-runner-tests</id>
-                <phase>integration-test</phase>
-                <goals>
-                  <goal>test</goal>
-                </goals>
-                <configuration>
-                  <groups>org.apache.beam.sdk.testing.ValidatesRunner</groups>
-                  <excludedGroups>
-                    org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders,
-                    org.apache.beam.sdk.testing.UsesSetState,
-                    org.apache.beam.sdk.testing.UsesMapState,
-                    org.apache.beam.sdk.testing.UsesAttemptedMetrics,
-                    org.apache.beam.sdk.testing.UsesCommittedMetrics,
-                    org.apache.beam.sdk.testing.UsesTestStream,
-                    org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs
-                  </excludedGroups>
-                  <parallel>none</parallel>
-                  <failIfNoTests>true</failIfNoTests>
-                  <dependenciesToScan>
-                    <dependency>org.apache.beam:beam-sdks-java-core</dependency>
-                  </dependenciesToScan>
-                  <systemPropertyVariables>
-                    <beamTestPipelineOptions>
-                      [
-                        "--runner=TestFlinkRunner",
-                        "--streaming=true"
-                      ]
-                    </beamTestPipelineOptions>
-                  </systemPropertyVariables>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-      </plugin>
-
-      <!-- Integration Tests -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-failsafe-plugin</artifactId>
-      </plugin>
-
-      <!-- Unit Tests -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-      </plugin>
-    </plugins>
-  </build>
-
-  <dependencies>
-    <!-- Flink dependencies -->
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-java</artifactId>
-      <version>${flink.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-clients_2.10</artifactId>
-      <version>${flink.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-runtime_2.10</artifactId>
-      <version>${flink.version}</version>
-    </dependency>
-
-    <!-- For testing -->
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-core</artifactId>
-      <version>${flink.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-runtime_2.10</artifactId>
-      <version>${flink.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-
-    <!-- Beam -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-core</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-core-java</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-core-construction-java</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-    </dependency>
-
-    <!--
-    Force an upgrade on the version of Apache Commons from Flink to support DEFLATE compression.
-    -->
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-compress</artifactId>
-      <scope>runtime</scope>
-    </dependency>
-
-    <!-- Test scoped -->
-    <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-bigquery</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-lang3</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <!-- Depend on test jar to scan for ValidatesRunner tests -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-core</artifactId>
-      <classifier>tests</classifier>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-streaming-java_2.10</artifactId>
-      <version>${flink.version}</version>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-test-utils_2.10</artifactId>
-      <version>${flink.version}</version>
-      <scope>test</scope>
-        <exclusions>
-            <exclusion>
-                <artifactId>apacheds-jdbm1</artifactId>
-                <groupId>org.apache.directory.jdbm</groupId>
-            </exclusion>
-        </exclusions>
-    </dependency>
-
-    <!-- Optional Pipeline Registration -->
-    <dependency>
-      <groupId>com.google.auto.service</groupId>
-      <artifactId>auto-service</artifactId>
-      <optional>true</optional>
-    </dependency>
-
-    <!-- transitive test dependencies from beam-sdk-java-core -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.dataformat</groupId>
-      <artifactId>jackson-dataformat-yaml</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-common-fn-api</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java
deleted file mode 100644
index b745f0b..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.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 org.apache.beam.runners.flink;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
-
-/**
- * {@link DefaultValueFactory} for getting a default value for the parallelism option
- * on {@link FlinkPipelineOptions}.
- *
- * <p>This will return either the default value from {@link GlobalConfiguration} or {@code 1}.
- * A valid {@link GlobalConfiguration} is only available if the program is executed by the Flink
- * run scripts.
- */
-public class DefaultParallelismFactory implements DefaultValueFactory<Integer> {
-  @Override
-  public Integer create(PipelineOptions options) {
-    return GlobalConfiguration.loadConfiguration()
-        .getInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, 1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
deleted file mode 100644
index 854b674..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.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 org.apache.beam.runners.flink;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a
- * Flink batch job.
- */
-class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
-
-  private static final Logger LOG = LoggerFactory.getLogger(FlinkBatchPipelineTranslator.class);
-
-  /**
-   * The necessary context in the case of a batch job.
-   */
-  private final FlinkBatchTranslationContext batchContext;
-
-  private int depth = 0;
-
-  public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
-    this.batchContext = new FlinkBatchTranslationContext(env, options);
-  }
-
-  @Override
-  @SuppressWarnings("rawtypes, unchecked")
-  public void translate(Pipeline pipeline) {
-    super.translate(pipeline);
-
-    // terminate dangling DataSets
-    for (DataSet<?> dataSet: batchContext.getDanglingDataSets().values()) {
-      dataSet.output(new DiscardingOutputFormat());
-    }
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline Visitor Methods
-  // --------------------------------------------------------------------------------------------
-
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    LOG.info("{} enterCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
-    this.depth++;
-
-    BatchTransformTranslator<?> translator = getTranslator(node);
-
-    if (translator != null) {
-      applyBatchTransform(node.getTransform(), node, translator);
-      LOG.info("{} translated- {}", genSpaces(this.depth), node.getFullName());
-      return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
-    } else {
-      return CompositeBehavior.ENTER_TRANSFORM;
-    }
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformHierarchy.Node node) {
-    this.depth--;
-    LOG.info("{} leaveCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
-  }
-
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    LOG.info("{} visitPrimitiveTransform- {}", genSpaces(this.depth), node.getFullName());
-
-    // get the transformation corresponding to the node we are
-    // currently visiting and translate it into its Flink alternative.
-    PTransform<?, ?> transform = node.getTransform();
-    BatchTransformTranslator<?> translator =
-        FlinkBatchTransformTranslators.getTranslator(transform);
-    if (translator == null) {
-      LOG.info(node.getTransform().getClass().toString());
-      throw new UnsupportedOperationException("The transform " + transform
-          + " is currently not supported.");
-    }
-    applyBatchTransform(transform, node, translator);
-  }
-
-  private <T extends PTransform<?, ?>> void applyBatchTransform(
-      PTransform<?, ?> transform,
-      TransformHierarchy.Node node,
-      BatchTransformTranslator<?> translator) {
-
-    @SuppressWarnings("unchecked")
-    T typedTransform = (T) transform;
-
-    @SuppressWarnings("unchecked")
-    BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
-
-    // create the applied PTransform on the batchContext
-    batchContext.setCurrentTransform(node.toAppliedPTransform());
-    typedTranslator.translateNode(typedTransform, batchContext);
-  }
-
-  /**
-   * A translator of a {@link PTransform}.
-   */
-  public interface BatchTransformTranslator<TransformT extends PTransform> {
-    void translateNode(TransformT transform, FlinkBatchTranslationContext context);
-  }
-
-  /**
-   * Returns a translator for the given node, if it is possible, otherwise null.
-   */
-  private static BatchTransformTranslator<?> getTranslator(TransformHierarchy.Node node) {
-    PTransform<?, ?> transform = node.getTransform();
-
-    // Root of the graph is null
-    if (transform == null) {
-      return null;
-    }
-
-    return FlinkBatchTransformTranslators.getTranslator(transform);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
deleted file mode 100644
index ff9521c..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
+++ /dev/null
@@ -1,723 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
-import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMergingPartialReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMergingReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkStatefulDoFnFunction;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.types.KvKeySelector;
-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.CoderRegistry;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.ListCoder;
-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.transforms.Combine;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.join.RawUnionValue;
-import org.apache.beam.sdk.transforms.join.UnionCoder;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
-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.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.Reshuffle;
-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 org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TupleTag;
-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.api.java.DataSet;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.operators.FlatMapOperator;
-import org.apache.flink.api.java.operators.GroupCombineOperator;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
-import org.apache.flink.api.java.operators.Grouping;
-import org.apache.flink.api.java.operators.MapPartitionOperator;
-import org.apache.flink.api.java.operators.SingleInputUdfOperator;
-import org.apache.flink.util.Collector;
-
-/**
- * Translators for transforming {@link PTransform PTransforms} to
- * Flink {@link DataSet DataSets}.
- */
-class FlinkBatchTransformTranslators {
-
-  // --------------------------------------------------------------------------------------------
-  //  Transform Translator Registry
-  // --------------------------------------------------------------------------------------------
-
-  @SuppressWarnings("rawtypes")
-  private static final Map<
-      Class<? extends PTransform>,
-      FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
-
-  static {
-    TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
-
-    TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
-    TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
-    TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorBatch());
-
-    TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslatorBatch());
-
-    TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch());
-
-    TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoTranslatorBatch());
-
-    TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
-  }
-
-
-  static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(
-      PTransform<?, ?> transform) {
-    return TRANSLATORS.get(transform.getClass());
-  }
-
-  private static class ReadSourceTranslatorBatch<T>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
-
-    @Override
-    public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
-      String name = transform.getName();
-      BoundedSource<T> source = transform.getSource();
-      PCollection<T> output = context.getOutput(transform);
-
-      TypeInformation<WindowedValue<T>> typeInformation = context.getTypeInfo(output);
-
-      DataSource<WindowedValue<T>> dataSource = new DataSource<>(
-          context.getExecutionEnvironment(),
-          new SourceInputFormat<>(source, context.getPipelineOptions()),
-          typeInformation,
-          name);
-
-      context.setOutputDataSet(output, dataSource);
-    }
-  }
-
-  private static class WindowAssignTranslatorBatch<T>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Window.Assign<T>> {
-
-    @Override
-    public void translateNode(Window.Assign<T> transform, FlinkBatchTranslationContext context) {
-      PValue input = context.getInput(transform);
-
-      TypeInformation<WindowedValue<T>> resultTypeInfo =
-          context.getTypeInfo(context.getOutput(transform));
-
-      DataSet<WindowedValue<T>> inputDataSet = context.getInputDataSet(input);
-
-      @SuppressWarnings("unchecked")
-      final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
-          (WindowingStrategy<T, ? extends BoundedWindow>)
-              context.getOutput(transform).getWindowingStrategy();
-
-      WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
-
-      FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
-          new FlinkAssignWindows<>(windowFn);
-
-      DataSet<WindowedValue<T>> resultDataSet = inputDataSet
-          .flatMap(assignWindowsFunction)
-          .name(context.getOutput(transform).getName())
-          .returns(resultTypeInfo);
-
-      context.setOutputDataSet(context.getOutput(transform), resultDataSet);
-    }
-  }
-
-  private static class GroupByKeyTranslatorBatch<K, InputT>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, InputT>> {
-
-    @Override
-    public void translateNode(
-        GroupByKey<K, InputT> transform,
-        FlinkBatchTranslationContext context) {
-
-      // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API
-      // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn
-
-      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
-          context.getInputDataSet(context.getInput(transform));
-
-      Combine.KeyedCombineFn<K, InputT, List<InputT>, List<InputT>> combineFn =
-          new Concatenate<InputT>().asKeyedFn();
-
-      KvCoder<K, InputT> inputCoder =
-          (KvCoder<K, InputT>) context.getInput(transform).getCoder();
-
-      Coder<List<InputT>> accumulatorCoder;
-
-      try {
-        accumulatorCoder =
-            combineFn.getAccumulatorCoder(
-                context.getInput(transform).getPipeline().getCoderRegistry(),
-                inputCoder.getKeyCoder(),
-                inputCoder.getValueCoder());
-      } catch (CannotProvideCoderException e) {
-        throw new RuntimeException(e);
-      }
-
-      WindowingStrategy<?, ?> windowingStrategy =
-          context.getInput(transform).getWindowingStrategy();
-
-      TypeInformation<WindowedValue<KV<K, List<InputT>>>> partialReduceTypeInfo =
-          new CoderTypeInformation<>(
-              WindowedValue.getFullCoder(
-                  KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
-                  windowingStrategy.getWindowFn().windowCoder()));
-
-
-      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
-          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
-
-      FlinkPartialReduceFunction<K, InputT, List<InputT>, ?> partialReduceFunction;
-      FlinkReduceFunction<K, List<InputT>, List<InputT>, ?> reduceFunction;
-
-      if (windowingStrategy.getWindowFn().isNonMerging()) {
-        @SuppressWarnings("unchecked")
-        WindowingStrategy<?, BoundedWindow> boundedStrategy =
-            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;
-
-        partialReduceFunction = new FlinkPartialReduceFunction<>(
-            combineFn,
-            boundedStrategy,
-            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
-            context.getPipelineOptions());
-
-        reduceFunction = new FlinkReduceFunction<>(
-            combineFn,
-            boundedStrategy,
-            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
-            context.getPipelineOptions());
-
-      } else {
-        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
-          throw new UnsupportedOperationException(
-              "Merging WindowFn with windows other than IntervalWindow are not supported.");
-        }
-
-        @SuppressWarnings("unchecked")
-        WindowingStrategy<?, IntervalWindow> intervalStrategy =
-            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;
-
-        partialReduceFunction = new FlinkMergingPartialReduceFunction<>(
-            combineFn,
-            intervalStrategy,
-            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
-            context.getPipelineOptions());
-
-        reduceFunction = new FlinkMergingReduceFunction<>(
-            combineFn,
-            intervalStrategy,
-            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
-            context.getPipelineOptions());
-      }
-
-      // Partially GroupReduce the values into the intermediate format AccumT (combine)
-      GroupCombineOperator<
-          WindowedValue<KV<K, InputT>>,
-          WindowedValue<KV<K, List<InputT>>>> groupCombine =
-          new GroupCombineOperator<>(
-              inputGrouping,
-              partialReduceTypeInfo,
-              partialReduceFunction,
-              "GroupCombine: " + transform.getName());
-
-      Grouping<WindowedValue<KV<K, List<InputT>>>> intermediateGrouping =
-          groupCombine.groupBy(new KvKeySelector<List<InputT>, K>(inputCoder.getKeyCoder()));
-
-      // Fully reduce the values and create output format VO
-      GroupReduceOperator<
-          WindowedValue<KV<K, List<InputT>>>, WindowedValue<KV<K, List<InputT>>>> outputDataSet =
-          new GroupReduceOperator<>(
-              intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName());
-
-      context.setOutputDataSet(context.getOutput(transform), outputDataSet);
-
-    }
-
-  }
-
-  private static class ReshuffleTranslatorBatch<K, InputT>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Reshuffle<K, InputT>> {
-
-    @Override
-    public void translateNode(
-        Reshuffle<K, InputT> transform,
-        FlinkBatchTranslationContext context) {
-
-      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
-          context.getInputDataSet(context.getInput(transform));
-
-      context.setOutputDataSet(context.getOutput(transform), inputDataSet.rebalance());
-
-    }
-
-  }
-
-  /**
-   * Combiner that combines {@code T}s into a single {@code List<T>} containing all inputs.
-   *
-   * <p>For internal use to translate {@link GroupByKey}. For a large {@link PCollection} this
-   * is expected to crash!
-   *
-   * <p>This is copied from the dataflow runner code.
-   *
-   * @param <T> the type of elements to concatenate.
-   */
-  private static class Concatenate<T> extends Combine.CombineFn<T, List<T>, List<T>> {
-    @Override
-    public List<T> createAccumulator() {
-      return new ArrayList<>();
-    }
-
-    @Override
-    public List<T> addInput(List<T> accumulator, T input) {
-      accumulator.add(input);
-      return accumulator;
-    }
-
-    @Override
-    public List<T> mergeAccumulators(Iterable<List<T>> accumulators) {
-      List<T> result = createAccumulator();
-      for (List<T> accumulator : accumulators) {
-        result.addAll(accumulator);
-      }
-      return result;
-    }
-
-    @Override
-    public List<T> extractOutput(List<T> accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<List<T>> getAccumulatorCoder(CoderRegistry registry, Coder<T> inputCoder) {
-      return ListCoder.of(inputCoder);
-    }
-
-    @Override
-    public Coder<List<T>> getDefaultOutputCoder(CoderRegistry registry, Coder<T> inputCoder) {
-      return ListCoder.of(inputCoder);
-    }
-  }
-
-
-  private static class CombinePerKeyTranslatorBatch<K, InputT, AccumT, OutputT>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
-          Combine.PerKey<K, InputT, OutputT>> {
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void translateNode(
-        Combine.PerKey<K, InputT, OutputT> transform,
-        FlinkBatchTranslationContext context) {
-      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
-          context.getInputDataSet(context.getInput(transform));
-
-      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> combineFn =
-          (CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT>) transform.getFn();
-
-      KvCoder<K, InputT> inputCoder =
-          (KvCoder<K, InputT>) context.getInput(transform).getCoder();
-
-      Coder<AccumT> accumulatorCoder;
-
-      try {
-        accumulatorCoder =
-            combineFn.getAccumulatorCoder(
-                context.getInput(transform).getPipeline().getCoderRegistry(),
-                inputCoder.getKeyCoder(),
-                inputCoder.getValueCoder());
-      } catch (CannotProvideCoderException e) {
-        throw new RuntimeException(e);
-      }
-
-      WindowingStrategy<?, ?> windowingStrategy =
-          context.getInput(transform).getWindowingStrategy();
-
-      TypeInformation<WindowedValue<KV<K, AccumT>>> partialReduceTypeInfo =
-          context.getTypeInfo(
-              KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
-              windowingStrategy);
-
-      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
-          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
-
-      // construct a map from side input to WindowingStrategy so that
-      // the DoFn runner can map main-input windows to side input windows
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
-      for (PCollectionView<?> sideInput: transform.getSideInputs()) {
-        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
-      }
-
-      if (windowingStrategy.getWindowFn().isNonMerging()) {
-        WindowingStrategy<?, BoundedWindow> boundedStrategy =
-            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;
-
-        FlinkPartialReduceFunction<K, InputT, AccumT, ?> partialReduceFunction =
-            new FlinkPartialReduceFunction<>(
-                combineFn,
-                boundedStrategy,
-                sideInputStrategies,
-                context.getPipelineOptions());
-
-        FlinkReduceFunction<K, AccumT, OutputT, ?> reduceFunction =
-            new FlinkReduceFunction<>(
-                combineFn,
-                boundedStrategy,
-                sideInputStrategies,
-                context.getPipelineOptions());
-
-        // Partially GroupReduce the values into the intermediate format AccumT (combine)
-        GroupCombineOperator<
-            WindowedValue<KV<K, InputT>>,
-            WindowedValue<KV<K, AccumT>>> groupCombine =
-            new GroupCombineOperator<>(
-                inputGrouping,
-                partialReduceTypeInfo,
-                partialReduceFunction,
-                "GroupCombine: " + transform.getName());
-
-        transformSideInputs(transform.getSideInputs(), groupCombine, context);
-
-        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
-            context.getTypeInfo(context.getOutput(transform));
-
-        Grouping<WindowedValue<KV<K, AccumT>>> intermediateGrouping =
-            groupCombine.groupBy(new KvKeySelector<AccumT, K>(inputCoder.getKeyCoder()));
-
-        // Fully reduce the values and create output format OutputT
-        GroupReduceOperator<
-            WindowedValue<KV<K, AccumT>>, WindowedValue<KV<K, OutputT>>> outputDataSet =
-            new GroupReduceOperator<>(
-                intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
-
-        transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
-        context.setOutputDataSet(context.getOutput(transform), outputDataSet);
-
-      } else {
-        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
-          throw new UnsupportedOperationException(
-              "Merging WindowFn with windows other than IntervalWindow are not supported.");
-        }
-
-        // for merging windows we can't to a pre-shuffle combine step since
-        // elements would not be in their correct windows for side-input access
-
-        WindowingStrategy<?, IntervalWindow> intervalStrategy =
-            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;
-
-        FlinkMergingNonShuffleReduceFunction<K, InputT, AccumT, OutputT, ?> reduceFunction =
-            new FlinkMergingNonShuffleReduceFunction<>(
-                combineFn,
-                intervalStrategy,
-                sideInputStrategies,
-                context.getPipelineOptions());
-
-        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
-            context.getTypeInfo(context.getOutput(transform));
-
-        Grouping<WindowedValue<KV<K, InputT>>> grouping =
-            inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
-
-        // Fully reduce the values and create output format OutputT
-        GroupReduceOperator<
-            WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, OutputT>>> outputDataSet =
-            new GroupReduceOperator<>(
-                grouping, reduceTypeInfo, reduceFunction, transform.getName());
-
-        transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
-        context.setOutputDataSet(context.getOutput(transform), outputDataSet);
-      }
-
-
-    }
-  }
-
-  private static void rejectSplittable(DoFn<?, ?> doFn) {
-    DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
-    if (signature.processElement().isSplittable()) {
-      throw new UnsupportedOperationException(
-          String.format(
-              "%s does not currently support splittable DoFn: %s",
-              FlinkRunner.class.getSimpleName(), doFn));
-    }
-  }
-
-  private static class ParDoTranslatorBatch<InputT, OutputT>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
-      ParDo.MultiOutput<InputT, OutputT>> {
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void translateNode(
-        ParDo.MultiOutput<InputT, OutputT> transform,
-        FlinkBatchTranslationContext context) {
-      DoFn<InputT, OutputT> doFn = transform.getFn();
-      rejectSplittable(doFn);
-      DataSet<WindowedValue<InputT>> inputDataSet =
-          context.getInputDataSet(context.getInput(transform));
-
-      Map<TupleTag<?>, PValue> outputs = context.getOutputs(transform);
-
-      Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
-      // put the main output at index 0, FlinkMultiOutputDoFnFunction  expects this
-      outputMap.put(transform.getMainOutputTag(), 0);
-      int count = 1;
-      for (TupleTag<?> tag : outputs.keySet()) {
-        if (!outputMap.containsKey(tag)) {
-          outputMap.put(tag, count++);
-        }
-      }
-
-      // assume that the windowing strategy is the same for all outputs
-      WindowingStrategy<?, ?> windowingStrategy = null;
-
-      // collect all output Coders and create a UnionCoder for our tagged outputs
-      List<Coder<?>> outputCoders = Lists.newArrayList();
-      for (PValue taggedValue : outputs.values()) {
-        checkState(
-            taggedValue instanceof PCollection,
-            "Within ParDo, got a non-PCollection output %s of type %s",
-            taggedValue,
-            taggedValue.getClass().getSimpleName());
-        PCollection<?> coll = (PCollection<?>) taggedValue;
-        outputCoders.add(coll.getCoder());
-        windowingStrategy = coll.getWindowingStrategy();
-      }
-
-      if (windowingStrategy == null) {
-        throw new IllegalStateException("No outputs defined.");
-      }
-
-      UnionCoder unionCoder = UnionCoder.of(outputCoders);
-
-      TypeInformation<WindowedValue<RawUnionValue>> typeInformation =
-          new CoderTypeInformation<>(
-              WindowedValue.getFullCoder(
-                  unionCoder,
-                  windowingStrategy.getWindowFn().windowCoder()));
-
-      List<PCollectionView<?>> sideInputs = transform.getSideInputs();
-
-      // construct a map from side input to WindowingStrategy so that
-      // the DoFn runner can map main-input windows to side input windows
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
-      for (PCollectionView<?> sideInput: sideInputs) {
-        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
-      }
-
-      SingleInputUdfOperator<WindowedValue<InputT>, WindowedValue<RawUnionValue>, ?> outputDataSet;
-      DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass());
-      if (signature.stateDeclarations().size() > 0
-          || signature.timerDeclarations().size() > 0) {
-
-        // Based on the fact that the signature is stateful, DoFnSignatures ensures
-        // that it is also keyed
-        KvCoder<?, InputT> inputCoder =
-            (KvCoder<?, InputT>) context.getInput(transform).getCoder();
-
-        FlinkStatefulDoFnFunction<?, ?, OutputT> doFnWrapper = new FlinkStatefulDoFnFunction<>(
-            (DoFn) doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(),
-            outputMap, transform.getMainOutputTag()
-        );
-
-        Grouping<WindowedValue<InputT>> grouping =
-            inputDataSet.groupBy(new KvKeySelector(inputCoder.getKeyCoder()));
-
-        outputDataSet =
-            new GroupReduceOperator(grouping, typeInformation, doFnWrapper, transform.getName());
-
-      } else {
-        FlinkDoFnFunction<InputT, RawUnionValue> doFnWrapper =
-            new FlinkDoFnFunction(
-                doFn,
-                windowingStrategy,
-                sideInputStrategies,
-                context.getPipelineOptions(),
-                outputMap,
-                transform.getMainOutputTag());
-
-        outputDataSet = new MapPartitionOperator<>(
-            inputDataSet, typeInformation,
-            doFnWrapper, transform.getName());
-
-      }
-
-      transformSideInputs(sideInputs, outputDataSet, context);
-
-      for (Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
-        pruneOutput(
-            outputDataSet,
-            context,
-            outputMap.get(output.getKey()),
-            (PCollection) output.getValue());
-      }
-
-    }
-
-    private <T> void pruneOutput(
-        DataSet<WindowedValue<RawUnionValue>> taggedDataSet,
-        FlinkBatchTranslationContext context,
-        int integerTag,
-        PCollection<T> collection) {
-      TypeInformation<WindowedValue<T>> outputType = context.getTypeInfo(collection);
-
-      FlinkMultiOutputPruningFunction<T> pruningFunction =
-          new FlinkMultiOutputPruningFunction<>(integerTag);
-
-      FlatMapOperator<WindowedValue<RawUnionValue>, WindowedValue<T>> pruningOperator =
-          new FlatMapOperator<>(
-              taggedDataSet,
-              outputType,
-              pruningFunction,
-              collection.getName());
-
-      context.setOutputDataSet(collection, pruningOperator);
-    }
-  }
-
-  private static class FlattenPCollectionTranslatorBatch<T>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
-      Flatten.PCollections<T>> {
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void translateNode(
-        Flatten.PCollections<T> transform,
-        FlinkBatchTranslationContext context) {
-
-      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
-      DataSet<WindowedValue<T>> result = null;
-
-      if (allInputs.isEmpty()) {
-
-        // create an empty dummy source to satisfy downstream operations
-        // we cannot create an empty source in Flink, therefore we have to
-        // add the flatMap that simply never forwards the single element
-        DataSource<String> dummySource =
-            context.getExecutionEnvironment().fromElements("dummy");
-        result = dummySource.flatMap(new FlatMapFunction<String, WindowedValue<T>>() {
-          @Override
-          public void flatMap(String s, Collector<WindowedValue<T>> collector) throws Exception {
-            // never return anything
-          }
-        }).returns(
-            new CoderTypeInformation<>(
-                WindowedValue.getFullCoder(
-                    (Coder<T>) VoidCoder.of(),
-                    GlobalWindow.Coder.INSTANCE)));
-      } else {
-        for (PValue taggedPc : allInputs.values()) {
-          checkArgument(
-              taggedPc instanceof PCollection,
-              "Got non-PCollection input to flatten: %s of type %s",
-              taggedPc,
-              taggedPc.getClass().getSimpleName());
-          PCollection<T> collection = (PCollection<T>) taggedPc;
-          DataSet<WindowedValue<T>> current = context.getInputDataSet(collection);
-          if (result == null) {
-            result = current;
-          } else {
-            result = result.union(current);
-          }
-        }
-      }
-
-      // insert a dummy filter, there seems to be a bug in Flink
-      // that produces duplicate elements after the union in some cases
-      // if we don't
-      result = result.filter(new FilterFunction<WindowedValue<T>>() {
-        @Override
-        public boolean filter(WindowedValue<T> tWindowedValue) throws Exception {
-          return true;
-        }
-      }).name("UnionFixFilter");
-      context.setOutputDataSet(context.getOutput(transform), result);
-    }
-  }
-
-  private static class CreatePCollectionViewTranslatorBatch<ElemT, ViewT>
-      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
-          View.CreatePCollectionView<ElemT, ViewT>> {
-
-    @Override
-    public void translateNode(
-        View.CreatePCollectionView<ElemT, ViewT> transform,
-        FlinkBatchTranslationContext context) {
-      DataSet<WindowedValue<ElemT>> inputDataSet =
-          context.getInputDataSet(context.getInput(transform));
-
-      PCollectionView<ViewT> input = transform.getView();
-
-      context.setSideInputDataSet(input, inputDataSet);
-    }
-  }
-
-  private static void transformSideInputs(
-      List<PCollectionView<?>> sideInputs,
-      SingleInputUdfOperator<?, ?, ?> outputDataSet,
-      FlinkBatchTranslationContext context) {
-    // get corresponding Flink broadcast DataSets
-    for (PCollectionView<?> input : sideInputs) {
-      DataSet<?> broadcastSet = context.getSideInputDataSet(input);
-      outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
-    }
-  }
-
-  private FlinkBatchTransformTranslators() {}
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
deleted file mode 100644
index 98dd0fb..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.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 org.apache.beam.runners.flink;
-
-import com.google.common.collect.Iterables;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.sdk.coders.Coder;
-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.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-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 org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * Helper for {@link FlinkBatchPipelineTranslator} and translators in
- * {@link FlinkBatchTransformTranslators}.
- */
-class FlinkBatchTranslationContext {
-
-  private final Map<PValue, DataSet<?>> dataSets;
-  private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
-
-  /**
-   * For keeping track about which DataSets don't have a successor. We
-   * need to terminate these with a discarding sink because the Beam
-   * model allows dangling operations.
-   */
-  private final Map<PValue, DataSet<?>> danglingDataSets;
-
-  private final ExecutionEnvironment env;
-  private final PipelineOptions options;
-
-  private AppliedPTransform<?, ?, ?> currentTransform;
-
-  // ------------------------------------------------------------------------
-
-  public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
-    this.env = env;
-    this.options = options;
-    this.dataSets = new HashMap<>();
-    this.broadcastDataSets = new HashMap<>();
-
-    this.danglingDataSets = new HashMap<>();
-  }
-
-  // ------------------------------------------------------------------------
-
-  public Map<PValue, DataSet<?>> getDanglingDataSets() {
-    return danglingDataSets;
-  }
-
-  public ExecutionEnvironment getExecutionEnvironment() {
-    return env;
-  }
-
-  public PipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> DataSet<WindowedValue<T>> getInputDataSet(PValue value) {
-    // assume that the DataSet is used as an input if retrieved here
-    danglingDataSets.remove(value);
-    return (DataSet<WindowedValue<T>>) dataSets.get(value);
-  }
-
-  public <T> void setOutputDataSet(PValue value, DataSet<WindowedValue<T>> set) {
-    if (!dataSets.containsKey(value)) {
-      dataSets.put(value, set);
-      danglingDataSets.put(value, set);
-    }
-  }
-
-  /**
-   * Sets the AppliedPTransform which carries input/output.
-   * @param currentTransform
-   */
-  public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
-    this.currentTransform = currentTransform;
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
-    return (DataSet<T>) broadcastDataSets.get(value);
-  }
-
-  public <ViewT, ElemT> void setSideInputDataSet(
-      PCollectionView<ViewT> value,
-      DataSet<WindowedValue<ElemT>> set) {
-    if (!broadcastDataSets.containsKey(value)) {
-      broadcastDataSets.put(value, set);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
-    return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy());
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(
-      Coder<T> coder,
-      WindowingStrategy<?, ?> windowingStrategy) {
-    WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
-        WindowedValue.getFullCoder(
-            coder,
-            windowingStrategy.getWindowFn().windowCoder());
-
-    return new CoderTypeInformation<>(windowedValueCoder);
-  }
-
-  Map<TupleTag<?>, PValue> getInputs(PTransform<?, ?> transform) {
-    return currentTransform.getInputs();
-  }
-
-  @SuppressWarnings("unchecked")
-  <T extends PValue> T getInput(PTransform<T, ?> transform) {
-    return (T) Iterables.getOnlyElement(currentTransform.getInputs().values());
-  }
-
-  Map<TupleTag<?>, PValue> getOutputs(PTransform<?, ?> transform) {
-    return currentTransform.getOutputs();
-  }
-
-  @SuppressWarnings("unchecked")
-  <T extends PValue> T getOutput(PTransform<?, T> transform) {
-    return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java
deleted file mode 100644
index bf4395f..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.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 org.apache.beam.runners.flink;
-
-import java.io.IOException;
-
-import org.apache.beam.sdk.AggregatorRetrievalException;
-import org.apache.beam.sdk.AggregatorValues;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.joda.time.Duration;
-
-
-/**
- * Result of a detached execution of a {@link org.apache.beam.sdk.Pipeline} with Flink.
- * In detached execution, results and job execution are currently unavailable.
- */
-public class FlinkDetachedRunnerResult implements PipelineResult {
-
-  FlinkDetachedRunnerResult() {}
-
-  @Override
-  public State getState() {
-    return State.UNKNOWN;
-  }
-
-  @Override
-  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
-      throws AggregatorRetrievalException {
-    throw new AggregatorRetrievalException(
-        "Accumulators can't be retrieved for detached Job executions.",
-        new UnsupportedOperationException());
-  }
-
-  @Override
-  public MetricResults metrics() {
-    throw new UnsupportedOperationException("The FlinkRunner does not currently support metrics.");
-  }
-
-  @Override
-  public State cancel() throws IOException {
-    throw new UnsupportedOperationException("Cancelling is not yet supported.");
-  }
-
-  @Override
-  public State waitUntilFinish() {
-    return State.UNKNOWN;
-  }
-
-  @Override
-  public State waitUntilFinish(Duration duration) {
-    return State.UNKNOWN;
-  }
-
-  @Override
-  public String toString() {
-    return "FlinkDetachedRunnerResult{}";
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index ba00036..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.List;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.CollectionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The class that instantiates and manages the execution of a given job.
- * Depending on if the job is a Streaming or Batch processing one, it creates
- * the adequate execution environment ({@link ExecutionEnvironment}
- * or {@link StreamExecutionEnvironment}), the necessary {@link FlinkPipelineTranslator}
- * ({@link FlinkBatchPipelineTranslator} or {@link FlinkStreamingPipelineTranslator}) to
- * transform the Beam job into a Flink one, and executes the (translated) job.
- */
-class FlinkPipelineExecutionEnvironment {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
-
-  private final FlinkPipelineOptions options;
-
-  /**
-   * The Flink Batch execution environment. This is instantiated to either a
-   * {@link org.apache.flink.api.java.CollectionEnvironment},
-   * a {@link org.apache.flink.api.java.LocalEnvironment} or
-   * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
-   * options.
-   */
-  private ExecutionEnvironment flinkBatchEnv;
-
-  /**
-   * The Flink Streaming execution environment. This is instantiated to either a
-   * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
-   * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
-   * on the configuration options, and more specifically, the url of the master.
-   */
-  private StreamExecutionEnvironment flinkStreamEnv;
-
-  /**
-   * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
-   * provided {@link FlinkPipelineOptions}.
-   *
-   * @param options the user-defined pipeline options.
-   * */
-  FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
-    this.options = checkNotNull(options);
-  }
-
-  /**
-   * 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 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.
-   * */
-  public void translate(FlinkRunner flinkRunner, Pipeline pipeline) {
-    this.flinkBatchEnv = null;
-    this.flinkStreamEnv = null;
-
-    PipelineTranslationOptimizer optimizer =
-        new PipelineTranslationOptimizer(TranslationMode.BATCH, options);
-
-    optimizer.translate(pipeline);
-    TranslationMode translationMode = optimizer.getTranslationMode();
-
-    FlinkPipelineTranslator translator;
-    if (translationMode == TranslationMode.STREAMING) {
-      this.flinkStreamEnv = createStreamExecutionEnvironment();
-      translator = new FlinkStreamingPipelineTranslator(flinkRunner, flinkStreamEnv, options);
-    } else {
-      this.flinkBatchEnv = createBatchExecutionEnvironment();
-      translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
-    }
-
-    translator.translate(pipeline);
-  }
-
-  /**
-   * Launches the program execution.
-   * */
-  public JobExecutionResult executePipeline() throws Exception {
-    final String jobName = options.getJobName();
-
-    if (flinkBatchEnv != null) {
-      return flinkBatchEnv.execute(jobName);
-    } else if (flinkStreamEnv != null) {
-      return flinkStreamEnv.execute(jobName);
-    } else {
-      throw new IllegalStateException("The Pipeline has not yet been translated.");
-    }
-  }
-
-  /**
-   * If the submitted job is a batch processing job, this method creates the adequate
-   * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
-   * on the user-specified options.
-   */
-  private ExecutionEnvironment createBatchExecutionEnvironment() {
-
-    LOG.info("Creating the required Batch Execution Environment.");
-
-    String masterUrl = options.getFlinkMaster();
-    ExecutionEnvironment flinkBatchEnv;
-
-    // depending on the master, create the right environment.
-    if (masterUrl.equals("[local]")) {
-      flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
-    } else if (masterUrl.equals("[collection]")) {
-      flinkBatchEnv = new CollectionEnvironment();
-    } else if (masterUrl.equals("[auto]")) {
-      flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
-    } else if (masterUrl.matches(".*:\\d*")) {
-      String[] parts = masterUrl.split(":");
-      List<String> stagingFiles = options.getFilesToStage();
-      flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
-          Integer.parseInt(parts[1]),
-          stagingFiles.toArray(new String[stagingFiles.size()]));
-    } else {
-      LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
-      flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
-    }
-
-    // set the correct parallelism.
-    if (options.getParallelism() != -1 && !(flinkBatchEnv instanceof CollectionEnvironment)) {
-      flinkBatchEnv.setParallelism(options.getParallelism());
-    }
-
-    // set parallelism in the options (required by some execution code)
-    options.setParallelism(flinkBatchEnv.getParallelism());
-
-    if (options.getObjectReuse()) {
-      flinkBatchEnv.getConfig().enableObjectReuse();
-    } else {
-      flinkBatchEnv.getConfig().disableObjectReuse();
-    }
-
-    return flinkBatchEnv;
-  }
-
-  /**
-   * If the submitted job is a stream processing job, this method creates the adequate
-   * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
-   * on the user-specified options.
-   */
-  private StreamExecutionEnvironment createStreamExecutionEnvironment() {
-
-    LOG.info("Creating the required Streaming Environment.");
-
-    String masterUrl = options.getFlinkMaster();
-    StreamExecutionEnvironment flinkStreamEnv = null;
-
-    // depending on the master, create the right environment.
-    if (masterUrl.equals("[local]")) {
-      flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
-    } else if (masterUrl.equals("[auto]")) {
-      flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-    } else if (masterUrl.matches(".*:\\d*")) {
-      String[] parts = masterUrl.split(":");
-      List<String> stagingFiles = options.getFilesToStage();
-      flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
-          Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
-    } else {
-      LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
-      flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-    }
-
-    // set the correct parallelism.
-    if (options.getParallelism() != -1) {
-      flinkStreamEnv.setParallelism(options.getParallelism());
-    }
-
-    // set parallelism in the options (required by some execution code)
-    options.setParallelism(flinkStreamEnv.getParallelism());
-
-    if (options.getObjectReuse()) {
-      flinkStreamEnv.getConfig().enableObjectReuse();
-    } else {
-      flinkStreamEnv.getConfig().disableObjectReuse();
-    }
-
-    // default to event time
-    flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-    // for the following 2 parameters, a value of -1 means that Flink will use
-    // the default values as specified in the configuration.
-    int numRetries = options.getNumberOfExecutionRetries();
-    if (numRetries != -1) {
-      flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
-    }
-    long retryDelay = options.getExecutionRetryDelay();
-    if (retryDelay != -1) {
-      flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
-    }
-
-    // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
-    // If the value is not -1, then the validity checks are applied.
-    // By default, checkpointing is disabled.
-    long checkpointInterval = options.getCheckpointingInterval();
-    if (checkpointInterval != -1) {
-      if (checkpointInterval < 1) {
-        throw new IllegalArgumentException("The checkpoint interval must be positive");
-      }
-      flinkStreamEnv.enableCheckpointing(checkpointInterval);
-    }
-
-    // State backend
-    final AbstractStateBackend stateBackend = options.getStateBackend();
-    if (stateBackend != null) {
-      flinkStreamEnv.setStateBackend(stateBackend);
-    }
-
-    return flinkStreamEnv;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index ef9afea..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.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 org.apache.beam.runners.flink;
-
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import java.util.List;
-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;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-
-/**
- * Options which can be used to configure a Flink PipelineRunner.
- */
-public interface FlinkPipelineOptions
-    extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
-
-  /**
-   * List of local files to make available to workers.
-   *
-   * <p>Jars are placed on the worker's classpath.
-   *
-   * <p>The default value is the list of jars from the main program's classpath.
-   */
-  @Description("Jar-Files to send to all workers and put on the classpath. "
-      + "The default value is all files from the classpath.")
-  @JsonIgnore
-  List<String> getFilesToStage();
-  void setFilesToStage(List<String> value);
-
-  /**
-   * The url of the Flink JobManager on which to execute pipelines. This can either be
-   * the the address of a cluster JobManager, in the form "host:port" or one of the special
-   * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
-   * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
-   * "[auto]" will let the system decide where to execute the pipeline based on the environment.
-   */
-  @Description("Address of the Flink Master where the Pipeline should be executed. Can"
-      + " either be of the form \"host:port\" or one of the special values [local], "
-      + "[collection] or [auto].")
-  String getFlinkMaster();
-  void setFlinkMaster(String value);
-
-  @Description("The degree of parallelism to be used when distributing operations onto workers.")
-  @Default.InstanceFactory(DefaultParallelismFactory.class)
-  Integer getParallelism();
-  void setParallelism(Integer value);
-
-  @Description("The interval between consecutive checkpoints (i.e. snapshots of the current"
-      + "pipeline state used for fault tolerance).")
-  @Default.Long(-1L)
-  Long getCheckpointingInterval();
-  void setCheckpointingInterval(Long interval);
-
-  @Description("Sets the number of times that failed tasks are re-executed. "
-      + "A value of zero effectively disables fault tolerance. A value of -1 indicates "
-      + "that the system default value (as defined in the configuration) should be used.")
-  @Default.Integer(-1)
-  Integer getNumberOfExecutionRetries();
-  void setNumberOfExecutionRetries(Integer retries);
-
-  @Description("Sets the delay between executions. A value of {@code -1} "
-      + "indicates that the default value should be used.")
-  @Default.Long(-1L)
-  Long getExecutionRetryDelay();
-  void setExecutionRetryDelay(Long delay);
-
-  @Description("Sets the behavior of reusing objects.")
-  @Default.Boolean(false)
-  Boolean getObjectReuse();
-  void setObjectReuse(Boolean reuse);
-
-  /**
-   * State backend to store Beam's state during computation.
-   * Note: Only applicable when executing in streaming mode.
-   */
-  @Description("Sets the state backend to use in streaming mode. "
-      + "Otherwise the default is read from the Flink config.")
-  @JsonIgnore
-  AbstractStateBackend getStateBackend();
-  void setStateBackend(AbstractStateBackend stateBackend);
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.java
deleted file mode 100644
index 65f416d..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineTranslator.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 org.apache.beam.runners.flink;
-
-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 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.
- */
-abstract class FlinkPipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
-
-  /**
-   * Translates the pipeline by passing this class as a visitor.
-   * @param pipeline The pipeline to be translated
-   */
-  public void translate(Pipeline pipeline) {
-    pipeline.traverseTopologically(this);
-  }
-
-  /**
-   * Utility formatting method.
-   * @param n number of spaces to generate
-   * @return String with "|" followed by n spaces
-   */
-  protected static String genSpaces(int n) {
-    StringBuilder builder = new StringBuilder();
-    for (int i = 0; i < n; i++) {
-      builder.append("|   ");
-    }
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
deleted file mode 100644
index 096f030..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.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 org.apache.beam.runners.flink;
-
-import com.google.common.base.Joiner;
-import java.io.File;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-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.TransformHierarchy;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.client.program.DetachedEnvironment;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to a Flink Plan and then executing them either locally
- * or on a Flink cluster, depending on the configuration.
- */
-public class FlinkRunner extends PipelineRunner<PipelineResult> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(FlinkRunner.class);
-
-  /**
-   * Provided options.
-   */
-  private final FlinkPipelineOptions options;
-
-  /**
-   * Construct a runner from the provided options.
-   *
-   * @param options Properties which configure the runner.
-   * @return The newly created runner.
-   */
-  public static FlinkRunner fromOptions(PipelineOptions options) {
-    FlinkPipelineOptions flinkOptions =
-        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
-    ArrayList<String> missing = new ArrayList<>();
-
-    if (flinkOptions.getAppName() == null) {
-      missing.add("appName");
-    }
-    if (missing.size() > 0) {
-      throw new IllegalArgumentException(
-          "Missing required values: " + Joiner.on(',').join(missing));
-    }
-
-    if (flinkOptions.getFilesToStage() == null) {
-      flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
-          FlinkRunner.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.",
-          flinkOptions.getFilesToStage().size());
-      LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
-    }
-
-    // Set Flink Master to [auto] if no option was specified.
-    if (flinkOptions.getFlinkMaster() == null) {
-      flinkOptions.setFlinkMaster("[auto]");
-    }
-
-    return new FlinkRunner(flinkOptions);
-  }
-
-  private FlinkRunner(FlinkPipelineOptions options) {
-    this.options = options;
-    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
-  }
-
-  @Override
-  public PipelineResult run(Pipeline pipeline) {
-    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
-
-    LOG.info("Executing pipeline using FlinkRunner.");
-
-    FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options);
-
-    LOG.info("Translating pipeline to Flink program.");
-    env.translate(this, pipeline);
-
-    JobExecutionResult result;
-    try {
-      LOG.info("Starting execution of Flink program.");
-      result = env.executePipeline();
-    } catch (Exception e) {
-      LOG.error("Pipeline execution failed", e);
-      throw new RuntimeException("Pipeline execution failed", e);
-    }
-
-    if (result instanceof DetachedEnvironment.DetachedJobExecutionResult) {
-      LOG.info("Pipeline submitted in Detached mode");
-      return new FlinkDetachedRunnerResult();
-    } else {
-      LOG.info("Execution finished in {} msecs", result.getNetRuntime());
-      Map<String, Object> accumulators = result.getAllAccumulatorResults();
-      if (accumulators != null && !accumulators.isEmpty()) {
-        LOG.info("Final aggregator values:");
-
-        for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
-          LOG.info("{} : {}", entry.getKey(), entry.getValue());
-        }
-      }
-
-      return new FlinkRunnerResult(accumulators, result.getNetRuntime());
-    }
-  }
-
-  /**
-   * For testing.
-   */
-  public FlinkPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public String toString() {
-    return "FlinkRunner#" + hashCode();
-  }
-
-  /**
-   * Attempts to detect all the resources the class loader has access to. This does not recurse
-   * to class loader parents stopping it from pulling in resources from the system class loader.
-   *
-   * @param classLoader The URLClassLoader to use to detect resources to stage.
-   * @return A list of absolute paths to the resources the class loader uses.
-   * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
-   *   of the resources the class loader exposes is not a file resource.
-   */
-  protected static List<String> detectClassPathResourcesToStage(
-      ClassLoader classLoader) {
-    if (!(classLoader instanceof URLClassLoader)) {
-      String message = String.format("Unable to use ClassLoader to detect classpath elements. "
-          + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
-      LOG.error(message);
-      throw new IllegalArgumentException(message);
-    }
-
-    List<String> files = new ArrayList<>();
-    for (URL url : ((URLClassLoader) classLoader).getURLs()) {
-      try {
-        files.add(new File(url.toURI()).getAbsolutePath());
-      } catch (IllegalArgumentException | URISyntaxException e) {
-        String message = String.format("Unable to convert url (%s) to file.", url);
-        LOG.error(message);
-        throw new IllegalArgumentException(message, e);
-      }
-    }
-    return files;
-  }
-
-  /** A set of {@link View}s with non-deterministic key coders. */
-  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
-
-  /**
-   * Records that the {@link PTransform} requires a deterministic key coder.
-   */
-  void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
-    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
-  }
-
-  /** 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 Pipeline.PipelineVisitor() {
-        @Override
-        public void visitValue(PValue value, TransformHierarchy.Node producer) {
-        }
-
-        @Override
-        public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-        }
-
-        @Override
-        public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-          return CompositeBehavior.ENTER_TRANSFORM;
-        }
-
-        @Override
-        public void leaveCompositeTransform(TransformHierarchy.Node 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 "
-          + "the Flink runner will require deterministic key coders.",
-          ptransformViewNamesWithNonDeterministicKeyCoders);
-    }
-  }
-}


[14/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
deleted file mode 100644
index 8f50105..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.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 org.apache.beam.runners.flink;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
-import org.apache.beam.sdk.PipelineResult;
-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.PipelineRunner;
-import org.apache.beam.sdk.util.UserCodeException;
-
-/**
- * Test Flink runner.
- */
-public class TestFlinkRunner extends PipelineRunner<PipelineResult> {
-
-  private FlinkRunner delegate;
-
-  private TestFlinkRunner(FlinkPipelineOptions options) {
-    // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment
-    options.setFlinkMaster("[auto]");
-    this.delegate = FlinkRunner.fromOptions(options);
-  }
-
-  public static TestFlinkRunner fromOptions(PipelineOptions options) {
-    FlinkPipelineOptions flinkOptions =
-        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
-    return new TestFlinkRunner(flinkOptions);
-  }
-
-  public static TestFlinkRunner create(boolean streaming) {
-    FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
-    flinkOptions.setRunner(TestFlinkRunner.class);
-    flinkOptions.setStreaming(streaming);
-    return TestFlinkRunner.fromOptions(flinkOptions);
-  }
-
-  @Override
-  public PipelineResult run(Pipeline pipeline) {
-    try {
-      return delegate.run(pipeline);
-    } catch (Throwable t) {
-      // Special case hack to pull out assertion errors from PAssert; instead there should
-      // probably be a better story along the lines of UserCodeException.
-      UserCodeException innermostUserCodeException = null;
-      Throwable current = t;
-      for (; current.getCause() != null; current = current.getCause()) {
-        if (current instanceof UserCodeException) {
-          innermostUserCodeException = ((UserCodeException) current);
-        }
-      }
-      if (innermostUserCodeException != null) {
-        current = innermostUserCodeException.getCause();
-      }
-      if (current instanceof AssertionError) {
-        throw (AssertionError) current;
-      }
-      throw new PipelineExecutionException(current);
-    }
-  }
-
-  public PipelineOptions getPipelineOptions() {
-    return delegate.getPipelineOptions();
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TranslationMode.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
deleted file mode 100644
index ad54750..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TranslationMode.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 org.apache.beam.runners.flink;
-
-/**
- * The translation mode of the Beam Pipeline.
- */
-enum TranslationMode {
-
-  /** Uses the batch mode of Flink. */
-  BATCH,
-
-  /** Uses the streaming mode of Flink. */
-  STREAMING
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
deleted file mode 100644
index 57f1e59..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java
deleted file mode 100644
index fb2493b..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.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 org.apache.beam.runners.flink.translation.functions;
-
-import org.apache.beam.runners.core.AggregatorFactory;
-import org.apache.beam.runners.core.ExecutionContext;
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.flink.api.common.functions.RuntimeContext;
-
-/**
- * A {@link AggregatorFactory} for the Flink Batch Runner.
- */
-public class FlinkAggregatorFactory implements AggregatorFactory{
-
-  private final RuntimeContext runtimeContext;
-
-  public FlinkAggregatorFactory(RuntimeContext runtimeContext) {
-    this.runtimeContext = runtimeContext;
-  }
-
-  @Override
-  public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
-      Class<?> fnClass, ExecutionContext.StepContext stepContext, String aggregatorName,
-      Combine.CombineFn<InputT, AccumT, OutputT> combine) {
-    @SuppressWarnings("unchecked")
-    SerializableFnAggregatorWrapper<InputT, OutputT> result =
-        (SerializableFnAggregatorWrapper<InputT, OutputT>)
-            runtimeContext.getAccumulator(aggregatorName);
-
-    if (result == null) {
-      result = new SerializableFnAggregatorWrapper<>(combine);
-      runtimeContext.addAccumulator(aggregatorName, result);
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
deleted file mode 100644
index 447b1e5..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.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 org.apache.beam.runners.flink.translation.functions;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.collect.Iterables;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-
-/**
- * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for
- * Flink functions.
- */
-class FlinkAssignContext<InputT, W extends BoundedWindow>
-    extends WindowFn<InputT, W>.AssignContext {
-  private final WindowedValue<InputT> value;
-
-  FlinkAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
-    fn.super();
-    checkArgument(
-        Iterables.size(value.getWindows()) == 1,
-        String.format(
-            "%s passed to window assignment must be in a single window, but it was in %s: %s",
-            WindowedValue.class.getSimpleName(),
-            Iterables.size(value.getWindows()),
-            value.getWindows()));
-    this.value = value;
-  }
-
-  @Override
-  public InputT element() {
-    return value.getValue();
-  }
-
-  @Override
-  public Instant timestamp() {
-    return value.getTimestamp();
-  }
-
-  @Override
-  public BoundedWindow window() {
-    return Iterables.getOnlyElement(value.getWindows());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java
deleted file mode 100644
index c3a5095..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.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 org.apache.beam.runners.flink.translation.functions;
-
-import java.util.Collection;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Flink {@link FlatMapFunction} for implementing
- * {@link org.apache.beam.sdk.transforms.windowing.Window.Assign}.
- */
-public class FlinkAssignWindows<T, W extends BoundedWindow>
-    implements FlatMapFunction<WindowedValue<T>, WindowedValue<T>> {
-
-  private final WindowFn<T, W> windowFn;
-
-  public FlinkAssignWindows(WindowFn<T, W> windowFn) {
-    this.windowFn = windowFn;
-  }
-
-  @Override
-  public void flatMap(
-      WindowedValue<T> input, Collector<WindowedValue<T>> collector) throws Exception {
-    Collection<W> windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input));
-    for (W window: windows) {
-      collector.collect(
-          WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 51582af..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.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 org.apache.beam.runners.flink.translation.functions;
-
-import java.util.Collections;
-import java.util.Map;
-import org.apache.beam.runners.core.DoFnRunner;
-import org.apache.beam.runners.core.DoFnRunners;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-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.transforms.reflect.DoFnInvoker;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
-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.TupleTag;
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.Collector;
-
-/**
- * Encapsulates a {@link DoFn}
- * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
- *
- * <p>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.
- */
-public class FlinkDoFnFunction<InputT, OutputT>
-    extends RichMapPartitionFunction<WindowedValue<InputT>, WindowedValue<OutputT>> {
-
-  private final SerializedPipelineOptions serializedOptions;
-
-  private final DoFn<InputT, OutputT> doFn;
-  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-
-  private final WindowingStrategy<?, ?> windowingStrategy;
-
-  private final Map<TupleTag<?>, Integer> outputMap;
-  private final TupleTag<OutputT> mainOutputTag;
-
-  private transient DoFnInvoker<InputT, OutputT> doFnInvoker;
-
-  public FlinkDoFnFunction(
-      DoFn<InputT, OutputT> doFn,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions options,
-      Map<TupleTag<?>, Integer> outputMap,
-      TupleTag<OutputT> mainOutputTag) {
-
-    this.doFn = doFn;
-    this.sideInputs = sideInputs;
-    this.serializedOptions = new SerializedPipelineOptions(options);
-    this.windowingStrategy = windowingStrategy;
-    this.outputMap = outputMap;
-    this.mainOutputTag = mainOutputTag;
-
-  }
-
-  @Override
-  public void mapPartition(
-      Iterable<WindowedValue<InputT>> values,
-      Collector<WindowedValue<OutputT>> out) throws Exception {
-
-    RuntimeContext runtimeContext = getRuntimeContext();
-
-    DoFnRunners.OutputManager outputManager;
-    if (outputMap == null) {
-      outputManager = new FlinkDoFnFunction.DoFnOutputManager(out);
-    } else {
-      // it has some additional outputs
-      outputManager =
-          new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap);
-    }
-
-    DoFnRunner<InputT, OutputT> doFnRunner = DoFnRunners.simpleRunner(
-        serializedOptions.getPipelineOptions(), doFn,
-        new FlinkSideInputReader(sideInputs, runtimeContext),
-        outputManager,
-        mainOutputTag,
-        // see SimpleDoFnRunner, just use it to limit number of additional outputs
-        Collections.<TupleTag<?>>emptyList(),
-        new FlinkNoOpStepContext(),
-        new FlinkAggregatorFactory(runtimeContext),
-        windowingStrategy);
-
-    doFnRunner.startBundle();
-
-    for (WindowedValue<InputT> value : values) {
-      doFnRunner.processElement(value);
-    }
-
-    doFnRunner.finishBundle();
-  }
-
-  @Override
-  public void open(Configuration parameters) throws Exception {
-    doFnInvoker = DoFnInvokers.invokerFor(doFn);
-    doFnInvoker.invokeSetup();
-  }
-
-  @Override
-  public void close() throws Exception {
-    doFnInvoker.invokeTeardown();
-  }
-
-  static class DoFnOutputManager
-      implements DoFnRunners.OutputManager {
-
-    private Collector collector;
-
-    DoFnOutputManager(Collector collector) {
-      this.collector = collector;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-      collector.collect(output);
-    }
-  }
-
-  static class MultiDoFnOutputManager
-      implements DoFnRunners.OutputManager {
-
-    private Collector<WindowedValue<RawUnionValue>> collector;
-    private Map<TupleTag<?>, Integer> outputMap;
-
-    MultiDoFnOutputManager(Collector<WindowedValue<RawUnionValue>> collector,
-                      Map<TupleTag<?>, Integer> outputMap) {
-      this.collector = collector;
-      this.outputMap = outputMap;
-    }
-
-    @Override
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-      collector.collect(WindowedValue.of(new RawUnionValue(outputMap.get(tag), output.getValue()),
-          output.getTimestamp(), output.getWindows(), output.getPane()));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
deleted file mode 100644
index 26fd0b4..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.functions;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.PerKeyCombineFnRunner;
-import org.apache.beam.runners.core.PerKeyCombineFnRunners;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.PaneInfo;
-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.PCollectionView;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * Special version of {@link FlinkReduceFunction} that supports merging windows. This
- * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
- * same behaviour as {@code MergeOverlappingIntervalWindows}.
- *
- * <p>This is different from the pair of function for the non-merging windows case
- * in that we cannot do combining before the shuffle because elements would not
- * yet be in their correct windows for side-input access.
- */
-public class FlinkMergingNonShuffleReduceFunction<
-    K, InputT, AccumT, OutputT, W extends IntervalWindow>
-    extends RichGroupReduceFunction<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, OutputT>>> {
-
-  private final CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> combineFn;
-
-  private final WindowingStrategy<?, W> windowingStrategy;
-
-  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-
-  private final SerializedPipelineOptions serializedOptions;
-
-  public FlinkMergingNonShuffleReduceFunction(
-      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> keyedCombineFn,
-      WindowingStrategy<?, W> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions) {
-
-    this.combineFn = keyedCombineFn;
-
-    this.windowingStrategy = windowingStrategy;
-    this.sideInputs = sideInputs;
-
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-
-  }
-
-  @Override
-  public void reduce(
-      Iterable<WindowedValue<KV<K, InputT>>> elements,
-      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
-
-    PipelineOptions options = serializedOptions.getPipelineOptions();
-
-    FlinkSideInputReader sideInputReader =
-        new FlinkSideInputReader(sideInputs, getRuntimeContext());
-
-    PerKeyCombineFnRunner<K, InputT, AccumT, OutputT> combineFnRunner =
-        PerKeyCombineFnRunners.create(combineFn);
-
-    @SuppressWarnings("unchecked")
-    OutputTimeFn<? super BoundedWindow> outputTimeFn =
-        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
-
-    // get all elements so that we can sort them, has to fit into
-    // memory
-    // this seems very unprudent, but correct, for now
-    List<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
-    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
-      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
-        sortedInput.add(exploded);
-      }
-    }
-    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
-      @Override
-      public int compare(
-          WindowedValue<KV<K, InputT>> o1,
-          WindowedValue<KV<K, InputT>> o2) {
-        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
-            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
-      }
-    });
-
-    // merge windows, we have to do it in an extra pre-processing step and
-    // can't do it as we go since the window of early elements would not
-    // be correct when calling the CombineFn
-    mergeWindow(sortedInput);
-
-    // iterate over the elements that are sorted by window timestamp
-    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
-
-    // create accumulator using the first elements key
-    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
-    K key = currentValue.getValue().getKey();
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
-    InputT firstValue = currentValue.getValue().getValue();
-    AccumT accumulator =
-        combineFnRunner.createAccumulator(key, options, sideInputReader, currentValue.getWindows());
-    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
-        options, sideInputReader, currentValue.getWindows());
-
-    // we use this to keep track of the timestamps assigned by the OutputTimeFn
-    Instant windowTimestamp =
-        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
-
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
-      IntervalWindow nextWindow =
-          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-
-      if (currentWindow.equals(nextWindow)) {
-        // continue accumulating and merge windows
-
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-
-        windowTimestamp = outputTimeFn.combine(
-            windowTimestamp,
-            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
-
-      } else {
-        // emit the value that we currently have
-        out.collect(
-            WindowedValue.of(
-                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                    options, sideInputReader, currentValue.getWindows())),
-                windowTimestamp,
-                currentWindow,
-                PaneInfo.NO_FIRING));
-
-        currentWindow = nextWindow;
-        currentValue = nextValue;
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.createAccumulator(key,
-            options, sideInputReader, currentValue.getWindows());
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
-      }
-
-    }
-
-    // emit the final accumulator
-    out.collect(
-        WindowedValue.of(
-            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                options, sideInputReader, currentValue.getWindows())),
-            windowTimestamp,
-            currentWindow,
-            PaneInfo.NO_FIRING));
-  }
-
-  /**
-   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
-   * This replaces windows in the input list.
-   */
-  private void mergeWindow(List<WindowedValue<KV<K, InputT>>> elements) {
-    int currentStart = 0;
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
-
-    for (int i = 1; i < elements.size(); i++) {
-      WindowedValue<KV<K, InputT>> nextValue = elements.get(i);
-      IntervalWindow nextWindow =
-          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-      if (currentWindow.intersects(nextWindow)) {
-        // we continue
-        currentWindow = currentWindow.span(nextWindow);
-      } else {
-        // retrofit the merged window to all windows up to "currentStart"
-        for (int j = i - 1; j >= currentStart; j--) {
-          WindowedValue<KV<K, InputT>> value = elements.get(j);
-          elements.set(
-              j,
-              WindowedValue.of(
-                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-        }
-        currentStart = i;
-        currentWindow = nextWindow;
-      }
-    }
-    if (currentStart < elements.size() - 1) {
-      // we have to retrofit the last batch
-      for (int j = elements.size() - 1; j >= currentStart; j--) {
-        WindowedValue<KV<K, InputT>> value = elements.get(j);
-        elements.set(
-            j,
-            WindowedValue.of(
-                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
deleted file mode 100644
index c68f155..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.functions;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.PerKeyCombineFnRunner;
-import org.apache.beam.runners.core.PerKeyCombineFnRunners;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.PaneInfo;
-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.PCollectionView;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * Special version of {@link FlinkPartialReduceFunction} that supports merging windows. This
- * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
- * same behaviour as {@code MergeOverlappingIntervalWindows}.
- */
-public class FlinkMergingPartialReduceFunction<K, InputT, AccumT, W extends IntervalWindow>
-    extends FlinkPartialReduceFunction<K, InputT, AccumT, W> {
-
-  public FlinkMergingPartialReduceFunction(
-      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn,
-      WindowingStrategy<?, W> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions) {
-    super(combineFn, windowingStrategy, sideInputs, pipelineOptions);
-  }
-
-  @Override
-  public void combine(
-      Iterable<WindowedValue<KV<K, InputT>>> elements,
-      Collector<WindowedValue<KV<K, AccumT>>> out) throws Exception {
-
-    PipelineOptions options = serializedOptions.getPipelineOptions();
-
-    FlinkSideInputReader sideInputReader =
-        new FlinkSideInputReader(sideInputs, getRuntimeContext());
-
-    PerKeyCombineFnRunner<K, InputT, AccumT, ?> combineFnRunner =
-        PerKeyCombineFnRunners.create(combineFn);
-
-    @SuppressWarnings("unchecked")
-    OutputTimeFn<? super BoundedWindow> outputTimeFn =
-        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
-
-    // get all elements so that we can sort them, has to fit into
-    // memory
-    // this seems very unprudent, but correct, for now
-    List<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
-    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
-      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
-        sortedInput.add(exploded);
-      }
-    }
-    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
-      @Override
-      public int compare(
-          WindowedValue<KV<K, InputT>> o1,
-          WindowedValue<KV<K, InputT>> o2) {
-        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
-            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
-      }
-    });
-
-    // merge windows, we have to do it in an extra pre-processing step and
-    // can't do it as we go since the window of early elements would not
-    // be correct when calling the CombineFn
-    mergeWindow(sortedInput);
-
-    // iterate over the elements that are sorted by window timestamp
-    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
-
-    // create accumulator using the first elements key
-    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
-    K key = currentValue.getValue().getKey();
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
-    InputT firstValue = currentValue.getValue().getValue();
-    AccumT accumulator = combineFnRunner.createAccumulator(key,
-        options, sideInputReader, currentValue.getWindows());
-    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
-        options, sideInputReader, currentValue.getWindows());
-
-    // we use this to keep track of the timestamps assigned by the OutputTimeFn
-    Instant windowTimestamp =
-        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
-
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
-      IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-
-      if (currentWindow.equals(nextWindow)) {
-        // continue accumulating and merge windows
-
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-
-        windowTimestamp = outputTimeFn.combine(
-            windowTimestamp,
-            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
-
-      } else {
-        // emit the value that we currently have
-        out.collect(
-            WindowedValue.of(
-                KV.of(key, accumulator),
-                windowTimestamp,
-                currentWindow,
-                PaneInfo.NO_FIRING));
-
-        currentWindow = nextWindow;
-        currentValue = nextValue;
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.createAccumulator(key,
-            options, sideInputReader, currentValue.getWindows());
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
-      }
-    }
-
-    // emit the final accumulator
-    out.collect(
-        WindowedValue.of(
-            KV.of(key, accumulator),
-            windowTimestamp,
-            currentWindow,
-            PaneInfo.NO_FIRING));
-  }
-
-  /**
-   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
-   * This replaces windows in the input list.
-   */
-  private void mergeWindow(List<WindowedValue<KV<K, InputT>>> elements) {
-    int currentStart = 0;
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
-
-    for (int i = 1; i < elements.size(); i++) {
-      WindowedValue<KV<K, InputT>> nextValue = elements.get(i);
-      IntervalWindow nextWindow =
-          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-      if (currentWindow.intersects(nextWindow)) {
-        // we continue
-        currentWindow = currentWindow.span(nextWindow);
-      } else {
-        // retrofit the merged window to all windows up to "currentStart"
-        for (int j = i - 1; j >= currentStart; j--) {
-          WindowedValue<KV<K, InputT>> value = elements.get(j);
-          elements.set(
-              j,
-              WindowedValue.of(
-                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-        }
-        currentStart = i;
-        currentWindow = nextWindow;
-      }
-    }
-    if (currentStart < elements.size() - 1) {
-      // we have to retrofit the last batch
-      for (int j = elements.size() - 1; j >= currentStart; j--) {
-        WindowedValue<KV<K, InputT>> value = elements.get(j);
-        elements.set(
-            j,
-            WindowedValue.of(
-                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java
deleted file mode 100644
index 84b3adc..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.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 org.apache.beam.runners.flink.translation.functions;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.PerKeyCombineFnRunner;
-import org.apache.beam.runners.core.PerKeyCombineFnRunners;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.PaneInfo;
-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.PCollectionView;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * Special version of {@link FlinkReduceFunction} that supports merging windows. This
- * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
- * same behaviour as {@code MergeOverlappingIntervalWindows}.
- */
-public class FlinkMergingReduceFunction<K, AccumT, OutputT, W extends IntervalWindow>
-    extends FlinkReduceFunction<K, AccumT, OutputT, W> {
-
-  public FlinkMergingReduceFunction(
-      CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> keyedCombineFn,
-      WindowingStrategy<?, W> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions) {
-    super(keyedCombineFn, windowingStrategy, sideInputs, pipelineOptions);
-  }
-
-  @Override
-  public void reduce(
-      Iterable<WindowedValue<KV<K, AccumT>>> elements,
-      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
-
-    PipelineOptions options = serializedOptions.getPipelineOptions();
-
-    FlinkSideInputReader sideInputReader =
-        new FlinkSideInputReader(sideInputs, getRuntimeContext());
-
-    PerKeyCombineFnRunner<K, ?, AccumT, OutputT> combineFnRunner =
-        PerKeyCombineFnRunners.create(combineFn);
-
-    @SuppressWarnings("unchecked")
-    OutputTimeFn<? super BoundedWindow> outputTimeFn =
-        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
-
-    // get all elements so that we can sort them, has to fit into
-    // memory
-    // this seems very unprudent, but correct, for now
-    ArrayList<WindowedValue<KV<K, AccumT>>> sortedInput = Lists.newArrayList();
-    for (WindowedValue<KV<K, AccumT>> inputValue : elements) {
-      for (WindowedValue<KV<K, AccumT>> exploded : inputValue.explodeWindows()) {
-        sortedInput.add(exploded);
-      }
-    }
-    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, AccumT>>>() {
-      @Override
-      public int compare(
-          WindowedValue<KV<K, AccumT>> o1,
-          WindowedValue<KV<K, AccumT>> o2) {
-        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
-            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
-      }
-    });
-
-    // merge windows, we have to do it in an extra pre-processing step and
-    // can't do it as we go since the window of early elements would not
-    // be correct when calling the CombineFn
-    mergeWindow(sortedInput);
-
-    // iterate over the elements that are sorted by window timestamp
-    final Iterator<WindowedValue<KV<K, AccumT>>> iterator = sortedInput.iterator();
-
-    // get the first accumulator
-    WindowedValue<KV<K, AccumT>> currentValue = iterator.next();
-    K key = currentValue.getValue().getKey();
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
-    AccumT accumulator = currentValue.getValue().getValue();
-
-    // we use this to keep track of the timestamps assigned by the OutputTimeFn,
-    // in FlinkPartialReduceFunction we already merge the timestamps assigned
-    // to individual elements, here we just merge them
-    List<Instant> windowTimestamps = new ArrayList<>();
-    windowTimestamps.add(currentValue.getTimestamp());
-
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, AccumT>> nextValue = iterator.next();
-      IntervalWindow nextWindow =
-          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-
-      if (nextWindow.equals(currentWindow)) {
-        // continue accumulating and merge windows
-
-        accumulator = combineFnRunner.mergeAccumulators(
-            key, ImmutableList.of(accumulator, nextValue.getValue().getValue()),
-            options, sideInputReader, currentValue.getWindows());
-
-        windowTimestamps.add(nextValue.getTimestamp());
-      } else {
-        out.collect(
-            WindowedValue.of(
-                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                    options, sideInputReader, currentValue.getWindows())),
-                outputTimeFn.merge(currentWindow, windowTimestamps),
-                currentWindow,
-                PaneInfo.NO_FIRING));
-
-        windowTimestamps.clear();
-
-        currentWindow = nextWindow;
-        currentValue = nextValue;
-        accumulator = nextValue.getValue().getValue();
-        windowTimestamps.add(nextValue.getTimestamp());
-      }
-    }
-
-    // emit the final accumulator
-    out.collect(
-        WindowedValue.of(
-            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                options, sideInputReader, currentValue.getWindows())),
-            outputTimeFn.merge(currentWindow, windowTimestamps),
-            currentWindow,
-            PaneInfo.NO_FIRING));
-  }
-
-  /**
-   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
-   * This replaces windows in the input list.
-   */
-  private void mergeWindow(List<WindowedValue<KV<K, AccumT>>> elements) {
-    int currentStart = 0;
-    IntervalWindow currentWindow =
-        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
-
-    for (int i = 1; i < elements.size(); i++) {
-      WindowedValue<KV<K, AccumT>> nextValue = elements.get(i);
-      IntervalWindow nextWindow =
-          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
-      if (currentWindow.intersects(nextWindow)) {
-        // we continue
-        currentWindow = currentWindow.span(nextWindow);
-      } else {
-        // retrofit the merged window to all windows up to "currentStart"
-        for (int j = i - 1; j >= currentStart; j--) {
-          WindowedValue<KV<K, AccumT>> value = elements.get(j);
-          elements.set(
-              j,
-              WindowedValue.of(
-                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-        }
-        currentStart = i;
-        currentWindow = nextWindow;
-      }
-    }
-    if (currentStart < elements.size() - 1) {
-      // we have to retrofit the last batch
-      for (int j = elements.size() - 1; j >= currentStart; j--) {
-        WindowedValue<KV<K, AccumT>> value = elements.get(j);
-        elements.set(
-            j,
-            WindowedValue.of(
-                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 9071cc5..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.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 org.apache.beam.runners.flink.translation.functions;
-
-import org.apache.beam.sdk.transforms.join.RawUnionValue;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * A {@link FlatMapFunction} function that filters out those elements that don't belong in this
- * output. We need this to implement MultiOutput ParDo functions in combination with
- * {@link FlinkDoFnFunction}.
- */
-public class FlinkMultiOutputPruningFunction<T>
-    implements FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<T>> {
-
-  private final int ourOutputTag;
-
-  public FlinkMultiOutputPruningFunction(int ourOutputTag) {
-    this.ourOutputTag = ourOutputTag;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void flatMap(
-      WindowedValue<RawUnionValue> windowedValue,
-      Collector<WindowedValue<T>> collector) throws Exception {
-    int unionTag = windowedValue.getValue().getUnionTag();
-    if (unionTag == ourOutputTag) {
-      collector.collect(
-          (WindowedValue<T>) windowedValue.withValue(windowedValue.getValue().getValue()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java
deleted file mode 100644
index 847a00a..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.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 org.apache.beam.runners.flink.translation.functions;
-
-import java.io.IOException;
-import org.apache.beam.runners.core.ExecutionContext.StepContext;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * A {@link StepContext} for Flink Batch Runner execution.
- */
-public class FlinkNoOpStepContext implements StepContext {
-
-  @Override
-  public String getStepName() {
-    return null;
-  }
-
-  @Override
-  public String getTransformName() {
-    return null;
-  }
-
-  @Override
-  public void noteOutput(WindowedValue<?> output) {
-
-  }
-
-  @Override
-  public void noteOutput(TupleTag<?> tag, WindowedValue<?> output) {
-
-  }
-
-  @Override
-  public <T, W extends BoundedWindow> void writePCollectionViewData(
-      TupleTag<?> tag,
-      Iterable<WindowedValue<T>> data,
-      Coder<Iterable<WindowedValue<T>>> dataCoder,
-      W window,
-      Coder<W> windowCoder) throws IOException {
-  }
-
-  @Override
-  public StateInternals<?> stateInternals() {
-    return null;
-  }
-
-  @Override
-  public TimerInternals timerInternals() {
-    return null;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 1d1ff9f..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.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 org.apache.beam.runners.flink.translation.functions;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.beam.runners.core.PerKeyCombineFnRunner;
-import org.apache.beam.runners.core.PerKeyCombineFnRunners;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.flink.api.common.functions.RichGroupCombineFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * This is is the first step for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey}
- * on Flink. The second part is {@link FlinkReduceFunction}. This function performs a local
- * combine step before shuffling while the latter does the final combination after a shuffle.
- *
- * <p>The input to {@link #combine(Iterable, Collector)} are elements of the same key but
- * for different windows. We have to ensure that we only combine elements of matching
- * windows.
- */
-public class FlinkPartialReduceFunction<K, InputT, AccumT, W extends BoundedWindow>
-    extends RichGroupCombineFunction<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, AccumT>>> {
-
-  protected final CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn;
-
-  protected final WindowingStrategy<?, W> windowingStrategy;
-
-  protected final SerializedPipelineOptions serializedOptions;
-
-  protected final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-
-  public FlinkPartialReduceFunction(
-      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn,
-      WindowingStrategy<?, W> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions) {
-
-    this.combineFn = combineFn;
-    this.windowingStrategy = windowingStrategy;
-    this.sideInputs = sideInputs;
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-
-  }
-
-  @Override
-  public void combine(
-      Iterable<WindowedValue<KV<K, InputT>>> elements,
-      Collector<WindowedValue<KV<K, AccumT>>> out) throws Exception {
-
-    PipelineOptions options = serializedOptions.getPipelineOptions();
-
-    FlinkSideInputReader sideInputReader =
-        new FlinkSideInputReader(sideInputs, getRuntimeContext());
-
-    PerKeyCombineFnRunner<K, InputT, AccumT, ?> combineFnRunner =
-        PerKeyCombineFnRunners.create(combineFn);
-
-    @SuppressWarnings("unchecked")
-    OutputTimeFn<? super BoundedWindow> outputTimeFn =
-        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
-
-    // get all elements so that we can sort them, has to fit into
-    // memory
-    // this seems very unprudent, but correct, for now
-    ArrayList<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
-    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
-      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
-        sortedInput.add(exploded);
-      }
-    }
-    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
-      @Override
-      public int compare(
-          WindowedValue<KV<K, InputT>> o1,
-          WindowedValue<KV<K, InputT>> o2) {
-        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
-            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
-      }
-    });
-
-    // iterate over the elements that are sorted by window timestamp
-    //
-    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
-
-    // create accumulator using the first elements key
-    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
-    K key = currentValue.getValue().getKey();
-    BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null);
-    InputT firstValue = currentValue.getValue().getValue();
-    AccumT accumulator = combineFnRunner.createAccumulator(key,
-        options, sideInputReader, currentValue.getWindows());
-    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
-        options, sideInputReader, currentValue.getWindows());
-
-    // we use this to keep track of the timestamps assigned by the OutputTimeFn
-    Instant windowTimestamp =
-        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
-
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
-      BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows());
-
-      if (nextWindow.equals(currentWindow)) {
-        // continue accumulating
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-
-        windowTimestamp = outputTimeFn.combine(
-            windowTimestamp,
-            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
-
-      } else {
-        // emit the value that we currently have
-        out.collect(
-            WindowedValue.of(
-                KV.of(key, accumulator),
-                windowTimestamp,
-                currentWindow,
-                PaneInfo.NO_FIRING));
-
-        currentWindow = nextWindow;
-        currentValue = nextValue;
-        InputT value = nextValue.getValue().getValue();
-        accumulator = combineFnRunner.createAccumulator(key,
-            options, sideInputReader, currentValue.getWindows());
-        accumulator = combineFnRunner.addInput(key, accumulator, value,
-            options, sideInputReader, currentValue.getWindows());
-        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
-      }
-    }
-
-    // emit the final accumulator
-    out.collect(
-        WindowedValue.of(
-            KV.of(key, accumulator),
-            windowTimestamp,
-            currentWindow,
-            PaneInfo.NO_FIRING));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 3e4f742..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.functions;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.PerKeyCombineFnRunner;
-import org.apache.beam.runners.core.PerKeyCombineFnRunners;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.CombineFnBase;
-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.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * This is the second part for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey}
- * on Flink, the second part is {@link FlinkReduceFunction}. This function performs the final
- * combination of the pre-combined values after a shuffle.
- *
- * <p>The input to {@link #reduce(Iterable, Collector)} are elements of the same key but
- * for different windows. We have to ensure that we only combine elements of matching
- * windows.
- */
-public class FlinkReduceFunction<K, AccumT, OutputT, W extends BoundedWindow>
-    extends RichGroupReduceFunction<WindowedValue<KV<K, AccumT>>, WindowedValue<KV<K, OutputT>>> {
-
-  protected final CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> combineFn;
-
-  protected final WindowingStrategy<?, W> windowingStrategy;
-
-  protected final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-
-  protected final SerializedPipelineOptions serializedOptions;
-
-  public FlinkReduceFunction(
-      CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> keyedCombineFn,
-      WindowingStrategy<?, W> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions) {
-
-    this.combineFn = keyedCombineFn;
-
-    this.windowingStrategy = windowingStrategy;
-    this.sideInputs = sideInputs;
-
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-
-  }
-
-  @Override
-  public void reduce(
-      Iterable<WindowedValue<KV<K, AccumT>>> elements,
-      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
-
-    PipelineOptions options = serializedOptions.getPipelineOptions();
-
-    FlinkSideInputReader sideInputReader =
-        new FlinkSideInputReader(sideInputs, getRuntimeContext());
-
-    PerKeyCombineFnRunner<K, ?, AccumT, OutputT> combineFnRunner =
-        PerKeyCombineFnRunners.create(combineFn);
-
-    @SuppressWarnings("unchecked")
-    OutputTimeFn<? super BoundedWindow> outputTimeFn =
-        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
-
-
-    // get all elements so that we can sort them, has to fit into
-    // memory
-    // this seems very unprudent, but correct, for now
-    ArrayList<WindowedValue<KV<K, AccumT>>> sortedInput = Lists.newArrayList();
-    for (WindowedValue<KV<K, AccumT>> inputValue: elements) {
-      for (WindowedValue<KV<K, AccumT>> exploded: inputValue.explodeWindows()) {
-        sortedInput.add(exploded);
-      }
-    }
-    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, AccumT>>>() {
-      @Override
-      public int compare(
-          WindowedValue<KV<K, AccumT>> o1,
-          WindowedValue<KV<K, AccumT>> o2) {
-        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
-            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
-      }
-    });
-
-    // iterate over the elements that are sorted by window timestamp
-    //
-    final Iterator<WindowedValue<KV<K, AccumT>>> iterator = sortedInput.iterator();
-
-    // get the first accumulator
-    WindowedValue<KV<K, AccumT>> currentValue = iterator.next();
-    K key = currentValue.getValue().getKey();
-    BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null);
-    AccumT accumulator = currentValue.getValue().getValue();
-
-    // we use this to keep track of the timestamps assigned by the OutputTimeFn,
-    // in FlinkPartialReduceFunction we already merge the timestamps assigned
-    // to individual elements, here we just merge them
-    List<Instant> windowTimestamps = new ArrayList<>();
-    windowTimestamps.add(currentValue.getTimestamp());
-
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, AccumT>> nextValue = iterator.next();
-      BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows());
-
-      if (nextWindow.equals(currentWindow)) {
-        // continue accumulating
-        accumulator = combineFnRunner.mergeAccumulators(
-            key, ImmutableList.of(accumulator, nextValue.getValue().getValue()),
-            options, sideInputReader, currentValue.getWindows());
-
-        windowTimestamps.add(nextValue.getTimestamp());
-      } else {
-        // emit the value that we currently have
-        out.collect(
-            WindowedValue.of(
-                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                    options, sideInputReader, currentValue.getWindows())),
-                outputTimeFn.merge(currentWindow, windowTimestamps),
-                currentWindow,
-                PaneInfo.NO_FIRING));
-
-        windowTimestamps.clear();
-
-        currentWindow = nextWindow;
-        currentValue = nextValue;
-        accumulator = nextValue.getValue().getValue();
-        windowTimestamps.add(nextValue.getTimestamp());
-      }
-
-    }
-
-    // emit the final accumulator
-    out.collect(
-        WindowedValue.of(
-            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
-                options, sideInputReader, currentValue.getWindows())),
-            outputTimeFn.merge(currentWindow, windowTimestamps),
-            currentWindow,
-            PaneInfo.NO_FIRING));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
deleted file mode 100644
index c317182..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.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 org.apache.beam.runners.flink.translation.functions;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-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 org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.api.common.functions.RuntimeContext;
-
-/**
- * A {@link SideInputReader} for the Flink Batch Runner.
- */
-public class FlinkSideInputReader implements SideInputReader {
-
-  private final Map<TupleTag<?>, WindowingStrategy<?, ?>> sideInputs;
-
-  private RuntimeContext runtimeContext;
-
-  public FlinkSideInputReader(Map<PCollectionView<?>, WindowingStrategy<?, ?>> indexByView,
-                              RuntimeContext runtimeContext) {
-    sideInputs = new HashMap<>();
-    for (Map.Entry<PCollectionView<?>, WindowingStrategy<?, ?>> entry : indexByView.entrySet()) {
-      sideInputs.put(entry.getKey().getTagInternal(), entry.getValue());
-    }
-    this.runtimeContext = runtimeContext;
-  }
-
-  @Nullable
-  @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
-    checkNotNull(view, "View passed to sideInput cannot be null");
-    TupleTag<Iterable<WindowedValue<?>>> tag = view.getTagInternal();
-    checkNotNull(
-        sideInputs.get(tag),
-        "Side input for " + view + " not available.");
-
-    Map<BoundedWindow, T> sideInputs =
-        runtimeContext.getBroadcastVariableWithInitializer(
-            tag.getId(), new SideInputInitializer<>(view));
-    T result = sideInputs.get(window);
-    if (result == null) {
-      result = view.getViewFn().apply(Collections.<WindowedValue<?>>emptyList());
-    }
-    return result;
-  }
-
-  @Override
-  public <T> boolean contains(PCollectionView<T> view) {
-    return sideInputs.containsKey(view.getTagInternal());
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return sideInputs.isEmpty();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
deleted file mode 100644
index c8193d2..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.functions;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.beam.runners.core.DoFnRunner;
-import org.apache.beam.runners.core.DoFnRunners;
-import org.apache.beam.runners.core.InMemoryStateInternals;
-import org.apache.beam.runners.core.InMemoryTimerInternals;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-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.reflect.DoFnInvoker;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-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.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * A {@link RichGroupReduceFunction} for stateful {@link ParDo} in Flink Batch Runner.
- */
-public class FlinkStatefulDoFnFunction<K, V, OutputT>
-    extends RichGroupReduceFunction<WindowedValue<KV<K, V>>, WindowedValue<OutputT>> {
-
-  private final DoFn<KV<K, V>, OutputT> dofn;
-  private final WindowingStrategy<?, ?> windowingStrategy;
-  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-  private final SerializedPipelineOptions serializedOptions;
-  private final Map<TupleTag<?>, Integer> outputMap;
-  private final TupleTag<OutputT> mainOutputTag;
-  private transient DoFnInvoker doFnInvoker;
-
-  public FlinkStatefulDoFnFunction(
-      DoFn<KV<K, V>, OutputT> dofn,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      PipelineOptions pipelineOptions,
-      Map<TupleTag<?>, Integer> outputMap,
-      TupleTag<OutputT> mainOutputTag) {
-
-    this.dofn = dofn;
-    this.windowingStrategy = windowingStrategy;
-    this.sideInputs = sideInputs;
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-    this.outputMap = outputMap;
-    this.mainOutputTag = mainOutputTag;
-  }
-
-  @Override
-  public void reduce(
-      Iterable<WindowedValue<KV<K, V>>> values,
-      Collector<WindowedValue<OutputT>> out) throws Exception {
-    RuntimeContext runtimeContext = getRuntimeContext();
-
-    DoFnRunners.OutputManager outputManager;
-    if (outputMap == null) {
-      outputManager = new FlinkDoFnFunction.DoFnOutputManager(out);
-    } else {
-      // it has some additional Outputs
-      outputManager =
-          new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap);
-    }
-
-    final Iterator<WindowedValue<KV<K, V>>> iterator = values.iterator();
-
-    // get the first value, we need this for initializing the state internals with the key.
-    // we are guaranteed to have a first value, otherwise reduce() would not have been called.
-    WindowedValue<KV<K, V>> currentValue = iterator.next();
-    final K key = currentValue.getValue().getKey();
-
-    final InMemoryStateInternals<K> stateInternals = InMemoryStateInternals.forKey(key);
-
-    // Used with Batch, we know that all the data is available for this key. We can't use the
-    // timer manager from the context because it doesn't exist. So we create one and advance
-    // time to the end after processing all elements.
-    final InMemoryTimerInternals timerInternals = new InMemoryTimerInternals();
-    timerInternals.advanceProcessingTime(Instant.now());
-    timerInternals.advanceSynchronizedProcessingTime(Instant.now());
-
-    DoFnRunner<KV<K, V>, OutputT> doFnRunner = DoFnRunners.simpleRunner(
-        serializedOptions.getPipelineOptions(), dofn,
-        new FlinkSideInputReader(sideInputs, runtimeContext),
-        outputManager,
-        mainOutputTag,
-        // see SimpleDoFnRunner, just use it to limit number of additional outputs
-        Collections.<TupleTag<?>>emptyList(),
-        new FlinkNoOpStepContext() {
-          @Override
-          public StateInternals<?> stateInternals() {
-            return stateInternals;
-          }
-          @Override
-          public TimerInternals timerInternals() {
-            return timerInternals;
-          }
-        },
-        new FlinkAggregatorFactory(runtimeContext),
-        windowingStrategy);
-
-    doFnRunner.startBundle();
-
-    doFnRunner.processElement(currentValue);
-    while (iterator.hasNext()) {
-      currentValue = iterator.next();
-      doFnRunner.processElement(currentValue);
-    }
-
-    // Finish any pending windows by advancing the input watermark to infinity.
-    timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    // Finally, advance the processing time to infinity to fire any timers.
-    timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    timerInternals.advanceSynchronizedProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    fireEligibleTimers(timerInternals, doFnRunner);
-
-    doFnRunner.finishBundle();
-  }
-
-  private void fireEligibleTimers(
-      InMemoryTimerInternals timerInternals, DoFnRunner<KV<K, V>, OutputT> runner)
-      throws Exception {
-
-    while (true) {
-
-      TimerInternals.TimerData timer;
-      boolean hasFired = false;
-
-      while ((timer = timerInternals.removeNextEventTimer()) != null) {
-        hasFired = true;
-        fireTimer(timer, runner);
-      }
-      while ((timer = timerInternals.removeNextProcessingTimer()) != null) {
-        hasFired = true;
-        fireTimer(timer, runner);
-      }
-      while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) != null) {
-        hasFired = true;
-        fireTimer(timer, runner);
-      }
-      if (!hasFired) {
-        break;
-      }
-    }
-  }
-
-  private void fireTimer(
-      TimerInternals.TimerData timer, DoFnRunner<KV<K, V>, OutputT> doFnRunner) {
-    StateNamespace namespace = timer.getNamespace();
-    checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
-    BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
-    doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain());
-  }
-
-  @Override
-  public void open(Configuration parameters) throws Exception {
-    doFnInvoker = DoFnInvokers.invokerFor(dofn);
-    doFnInvoker.invokeSetup();
-  }
-
-  @Override
-  public void close() throws Exception {
-    doFnInvoker.invokeTeardown();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java
deleted file mode 100644
index 12222b4..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.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 org.apache.beam.runners.flink.translation.functions;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-
-/**
- * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map}
- * from window to side input.
- */
-public class SideInputInitializer<ElemT, ViewT, W extends BoundedWindow>
-    implements BroadcastVariableInitializer<WindowedValue<ElemT>, Map<BoundedWindow, ViewT>> {
-
-  PCollectionView<ViewT> view;
-
-  public SideInputInitializer(PCollectionView<ViewT> view) {
-    this.view = view;
-  }
-
-  @Override
-  public Map<BoundedWindow, ViewT> initializeBroadcastVariable(
-      Iterable<WindowedValue<ElemT>> inputValues) {
-
-    // first partition into windows
-    Map<BoundedWindow, List<WindowedValue<ElemT>>> partitionedElements = new HashMap<>();
-    for (WindowedValue<ElemT> value: inputValues) {
-      for (BoundedWindow window: value.getWindows()) {
-        List<WindowedValue<ElemT>> windowedValues = partitionedElements.get(window);
-        if (windowedValues == null) {
-          windowedValues = new ArrayList<>();
-          partitionedElements.put(window, windowedValues);
-        }
-        windowedValues.add(value);
-      }
-    }
-
-    Map<BoundedWindow, ViewT> resultMap = new HashMap<>();
-
-    for (Map.Entry<BoundedWindow, List<WindowedValue<ElemT>>> elements:
-        partitionedElements.entrySet()) {
-
-      @SuppressWarnings("unchecked")
-      Iterable<WindowedValue<?>> elementsIterable =
-          (List<WindowedValue<?>>) (List<?>) elements.getValue();
-
-      resultMap.put(elements.getKey(), view.getViewFn().apply(elementsIterable));
-    }
-
-    return resultMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
deleted file mode 100644
index 9f11212..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.functions;


[11/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
deleted file mode 100644
index 24b340e..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java
+++ /dev/null
@@ -1,487 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTag;
-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.ListCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-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.CoderUtils;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MapState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.SetState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.KeyGroupsList;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.streaming.api.operators.HeapInternalTimerService;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.Preconditions;
-
-/**
- * {@link StateInternals} that uses {@link KeyGroupCheckpointedOperator}
- * to checkpoint state.
- *
- * <p>Note:
- * Ignore index of key.
- * Just implement BagState.
- *
- * <p>Reference from {@link HeapInternalTimerService} to the local key-group range.
- */
-public class FlinkKeyGroupStateInternals<K> implements StateInternals<K> {
-
-  private final Coder<K> keyCoder;
-  private final KeyGroupsList localKeyGroupRange;
-  private KeyedStateBackend keyedStateBackend;
-  private final int localKeyGroupRangeStartIdx;
-
-  // stateName -> namespace -> (valueCoder, value)
-  private final Map<String, Tuple2<Coder<?>, Map<String, ?>>>[] stateTables;
-
-  public FlinkKeyGroupStateInternals(
-      Coder<K> keyCoder,
-      KeyedStateBackend keyedStateBackend) {
-    this.keyCoder = keyCoder;
-    this.keyedStateBackend = keyedStateBackend;
-    this.localKeyGroupRange = keyedStateBackend.getKeyGroupRange();
-    // find the starting index of the local key-group range
-    int startIdx = Integer.MAX_VALUE;
-    for (Integer keyGroupIdx : localKeyGroupRange) {
-      startIdx = Math.min(keyGroupIdx, startIdx);
-    }
-    this.localKeyGroupRangeStartIdx = startIdx;
-    stateTables = (Map<String, Tuple2<Coder<?>, Map<String, ?>>>[])
-        new Map[localKeyGroupRange.getNumberOfKeyGroups()];
-    for (int i = 0; i < stateTables.length; i++) {
-      stateTables[i] = new HashMap<>();
-    }
-  }
-
-  @Override
-  public K getKey() {
-    ByteBuffer keyBytes = (ByteBuffer) keyedStateBackend.getCurrentKey();
-    try {
-      return CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array());
-    } catch (CoderException e) {
-      throw new RuntimeException("Error decoding key.", e);
-    }
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address) {
-
-    return state(namespace, address, StateContexts.nullContext());
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address,
-      final StateContext<?> context) {
-
-    return address.bind(new StateTag.StateBinder<K>() {
-
-      @Override
-      public <T> ValueState<T> bindValue(
-          StateTag<? super K, ValueState<T>> address,
-          Coder<T> coder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", ValueState.class.getSimpleName()));
-      }
-
-      @Override
-      public <T> BagState<T> bindBag(
-          StateTag<? super K, BagState<T>> address,
-          Coder<T> elemCoder) {
-
-        return new FlinkKeyGroupBagState<>(address, namespace, elemCoder);
-      }
-
-      @Override
-      public <T> SetState<T> bindSet(
-          StateTag<? super K, SetState<T>> address,
-          Coder<T> elemCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", SetState.class.getSimpleName()));
-      }
-
-      @Override
-      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-          StateTag<? super K, MapState<KeyT, ValueT>> spec,
-          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", MapState.class.getSimpleName()));
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT>
-      bindCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException("bindCombiningValue is not supported.");
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported.");
-
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          CombineWithContext.KeyedCombineFnWithContext<
-              ? super K, InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException(
-            "bindKeyedCombiningValueWithContext is not supported.");
-      }
-
-      @Override
-      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
-          StateTag<? super K, WatermarkHoldState<W>> address,
-          OutputTimeFn<? super W> outputTimeFn) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", CombiningState.class.getSimpleName()));
-      }
-    });
-  }
-
-  /**
-   * Reference from {@link Combine.CombineFn}.
-   *
-   * <p>Accumulators are stored in each KeyGroup, call addInput() when a element comes,
-   * call extractOutput() to produce the desired value when need to read data.
-   */
-  interface KeyGroupCombiner<InputT, AccumT, OutputT> {
-
-    /**
-     * Returns a new, mutable accumulator value, representing the accumulation
-     * of zero input values.
-     */
-    AccumT createAccumulator();
-
-    /**
-     * Adds the given input value to the given accumulator, returning the
-     * new accumulator value.
-     */
-    AccumT addInput(AccumT accumulator, InputT input);
-
-    /**
-     * Returns the output value that is the result of all accumulators from KeyGroups
-     * that are assigned to this operator.
-     */
-    OutputT extractOutput(Iterable<AccumT> accumulators);
-  }
-
-  private abstract class AbstractKeyGroupState<InputT, AccumT, OutputT> {
-
-    private String stateName;
-    private String namespace;
-    private Coder<AccumT> coder;
-    private KeyGroupCombiner<InputT, AccumT, OutputT> keyGroupCombiner;
-
-    AbstractKeyGroupState(
-        String stateName,
-        String namespace,
-        Coder<AccumT> coder,
-        KeyGroupCombiner<InputT, AccumT, OutputT> keyGroupCombiner) {
-      this.stateName = stateName;
-      this.namespace = namespace;
-      this.coder = coder;
-      this.keyGroupCombiner = keyGroupCombiner;
-    }
-
-    /**
-     * Choose keyGroup of input and addInput to accumulator.
-     */
-    void addInput(InputT input) {
-      int keyGroupIdx = keyedStateBackend.getCurrentKeyGroupIndex();
-      int localIdx = getIndexForKeyGroup(keyGroupIdx);
-      Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
-      Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
-      if (tuple2 == null) {
-        tuple2 = new Tuple2<>();
-        tuple2.f0 = coder;
-        tuple2.f1 = new HashMap<>();
-        stateTable.put(stateName, tuple2);
-      }
-      Map<String, AccumT> map = (Map<String, AccumT>) tuple2.f1;
-      AccumT accumulator = map.get(namespace);
-      if (accumulator == null) {
-        accumulator = keyGroupCombiner.createAccumulator();
-      }
-      accumulator = keyGroupCombiner.addInput(accumulator, input);
-      map.put(namespace, accumulator);
-    }
-
-    /**
-     * Get all accumulators and invoke extractOutput().
-     */
-    OutputT extractOutput() {
-      List<AccumT> accumulators = new ArrayList<>(stateTables.length);
-      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
-        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
-        if (tuple2 != null) {
-          AccumT accumulator = (AccumT) tuple2.f1.get(namespace);
-          if (accumulator != null) {
-            accumulators.add(accumulator);
-          }
-        }
-      }
-      return keyGroupCombiner.extractOutput(accumulators);
-    }
-
-    /**
-     * Find the first accumulator and return immediately.
-     */
-    boolean isEmptyInternal() {
-      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
-        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
-        if (tuple2 != null) {
-          AccumT accumulator = (AccumT) tuple2.f1.get(namespace);
-          if (accumulator != null) {
-            return false;
-          }
-        }
-      }
-      return true;
-    }
-
-    /**
-     * Clear accumulators and clean empty map.
-     */
-    void clearInternal() {
-      for (Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable : stateTables) {
-        Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
-        if (tuple2 != null) {
-          tuple2.f1.remove(namespace);
-          if (tuple2.f1.size() == 0) {
-            stateTable.remove(stateName);
-          }
-        }
-      }
-    }
-
-  }
-
-  private int getIndexForKeyGroup(int keyGroupIdx) {
-    checkArgument(localKeyGroupRange.contains(keyGroupIdx),
-        "Key Group " + keyGroupIdx + " does not belong to the local range.");
-    return keyGroupIdx - this.localKeyGroupRangeStartIdx;
-  }
-
-  private class KeyGroupBagCombiner<T> implements KeyGroupCombiner<T, List<T>, Iterable<T>> {
-
-    @Override
-    public List<T> createAccumulator() {
-      return new ArrayList<>();
-    }
-
-    @Override
-    public List<T> addInput(List<T> accumulator, T input) {
-      accumulator.add(input);
-      return accumulator;
-    }
-
-    @Override
-    public Iterable<T> extractOutput(Iterable<List<T>> accumulators) {
-      List<T> result = new ArrayList<>();
-      // maybe can return an unmodifiable view.
-      for (List<T> list : accumulators) {
-        result.addAll(list);
-      }
-      return result;
-    }
-  }
-
-  private class FlinkKeyGroupBagState<T> extends AbstractKeyGroupState<T, List<T>, Iterable<T>>
-      implements BagState<T> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, BagState<T>> address;
-
-    FlinkKeyGroupBagState(
-        StateTag<? super K, BagState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-      super(address.getId(), namespace.stringKey(), ListCoder.of(coder),
-          new KeyGroupBagCombiner<T>());
-      this.namespace = namespace;
-      this.address = address;
-    }
-
-    @Override
-    public void add(T input) {
-      addInput(input);
-    }
-
-    @Override
-    public BagState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public Iterable<T> read() {
-      Iterable<T> result = extractOutput();
-      return result != null ? result : Collections.<T>emptyList();
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return isEmptyInternal();
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkKeyGroupBagState<?> that = (FlinkKeyGroupBagState<?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  /**
-   * Snapshots the state {@code (stateName -> (valueCoder && (namespace -> value)))} for a given
-   * {@code keyGroupIdx}.
-   *
-   * @param keyGroupIdx the id of the key-group to be put in the snapshot.
-   * @param out the stream to write to.
-   */
-  public void snapshotKeyGroupState(int keyGroupIdx, DataOutputStream out) throws Exception {
-    int localIdx = getIndexForKeyGroup(keyGroupIdx);
-    Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
-    Preconditions.checkState(stateTable.size() <= Short.MAX_VALUE,
-        "Too many States: " + stateTable.size() + ". Currently at most "
-            + Short.MAX_VALUE + " states are supported");
-    out.writeShort(stateTable.size());
-    for (Map.Entry<String, Tuple2<Coder<?>, Map<String, ?>>> entry : stateTable.entrySet()) {
-      out.writeUTF(entry.getKey());
-      Coder coder = entry.getValue().f0;
-      InstantiationUtil.serializeObject(out, coder);
-      Map<String, ?> map = entry.getValue().f1;
-      out.writeInt(map.size());
-      for (Map.Entry<String, ?> entry1 : map.entrySet()) {
-        StringUtf8Coder.of().encode(entry1.getKey(), out, Context.NESTED);
-        coder.encode(entry1.getValue(), out, Context.NESTED);
-      }
-    }
-  }
-
-  /**
-   * Restore the state {@code (stateName -> (valueCoder && (namespace -> value)))}
-   * for a given {@code keyGroupIdx}.
-   *
-   * @param keyGroupIdx the id of the key-group to be put in the snapshot.
-   * @param in the stream to read from.
-   * @param userCodeClassLoader the class loader that will be used to deserialize
-   *                            the valueCoder.
-   */
-  public void restoreKeyGroupState(int keyGroupIdx, DataInputStream in,
-                                   ClassLoader userCodeClassLoader) throws Exception {
-    int localIdx = getIndexForKeyGroup(keyGroupIdx);
-    Map<String, Tuple2<Coder<?>, Map<String, ?>>> stateTable = stateTables[localIdx];
-    int numStates = in.readShort();
-    for (int i = 0; i < numStates; ++i) {
-      String stateName = in.readUTF();
-      Coder coder = InstantiationUtil.deserializeObject(in, userCodeClassLoader);
-      Tuple2<Coder<?>, Map<String, ?>> tuple2 = stateTable.get(stateName);
-      if (tuple2 == null) {
-        tuple2 = new Tuple2<>();
-        tuple2.f0 = coder;
-        tuple2.f1 = new HashMap<>();
-        stateTable.put(stateName, tuple2);
-      }
-      Map<String, Object> map = (Map<String, Object>) tuple2.f1;
-      int mapSize = in.readInt();
-      for (int j = 0; j < mapSize; j++) {
-        String namespace = StringUtf8Coder.of().decode(in, Context.NESTED);
-        Object value = coder.decode(in, Context.NESTED);
-        map.put(namespace, value);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
deleted file mode 100644
index 2bf0bf1..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import com.google.common.collect.Iterators;
-import java.util.Collections;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.sdk.coders.Coder;
-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.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MapState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.SetState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.runtime.state.OperatorStateBackend;
-
-/**
- * {@link StateInternals} that uses a Flink {@link OperatorStateBackend}
- * to manage the split-distribute state.
- *
- * <p>Elements in ListState will be redistributed in round robin fashion
- * to operators when restarting with a different parallelism.
- *
- *  <p>Note:
- *  Ignore index of key and namespace.
- *  Just implement BagState.
- */
-public class FlinkSplitStateInternals<K> implements StateInternals<K> {
-
-  private final OperatorStateBackend stateBackend;
-
-  public FlinkSplitStateInternals(OperatorStateBackend stateBackend) {
-    this.stateBackend = stateBackend;
-  }
-
-  @Override
-  public K getKey() {
-    return null;
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address) {
-
-    return state(namespace, address, StateContexts.nullContext());
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address,
-      final StateContext<?> context) {
-
-    return address.bind(new StateTag.StateBinder<K>() {
-
-      @Override
-      public <T> ValueState<T> bindValue(
-          StateTag<? super K, ValueState<T>> address,
-          Coder<T> coder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", ValueState.class.getSimpleName()));
-      }
-
-      @Override
-      public <T> BagState<T> bindBag(
-          StateTag<? super K, BagState<T>> address,
-          Coder<T> elemCoder) {
-
-        return new FlinkSplitBagState<>(stateBackend, address, namespace, elemCoder);
-      }
-
-      @Override
-      public <T> SetState<T> bindSet(
-          StateTag<? super K, SetState<T>> address,
-          Coder<T> elemCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", SetState.class.getSimpleName()));
-      }
-
-      @Override
-      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-          StateTag<? super K, MapState<KeyT, ValueT>> spec,
-          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", MapState.class.getSimpleName()));
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT>
-      bindCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException("bindCombiningValue is not supported.");
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported.");
-
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          CombineWithContext.KeyedCombineFnWithContext<
-              ? super K, InputT, AccumT, OutputT> combineFn) {
-        throw new UnsupportedOperationException(
-            "bindKeyedCombiningValueWithContext is not supported.");
-      }
-
-      @Override
-      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
-          StateTag<? super K, WatermarkHoldState<W>> address,
-          OutputTimeFn<? super W> outputTimeFn) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", CombiningState.class.getSimpleName()));
-      }
-    });
-  }
-
-  private static class FlinkSplitBagState<K, T> implements BagState<T> {
-
-    private final ListStateDescriptor<T> descriptor;
-    private OperatorStateBackend flinkStateBackend;
-    private final StateNamespace namespace;
-    private final StateTag<? super K, BagState<T>> address;
-
-    FlinkSplitBagState(
-        OperatorStateBackend flinkStateBackend,
-        StateTag<? super K, BagState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-      this.flinkStateBackend = flinkStateBackend;
-      this.namespace = namespace;
-      this.address = address;
-
-      CoderTypeInformation<T> typeInfo =
-          new CoderTypeInformation<>(coder);
-
-      descriptor = new ListStateDescriptor<>(address.getId(),
-          typeInfo.createSerializer(new ExecutionConfig()));
-    }
-
-    @Override
-    public void add(T input) {
-      try {
-        flinkStateBackend.getOperatorState(descriptor).add(input);
-      } catch (Exception e) {
-        throw new RuntimeException("Error updating state.", e);
-      }
-    }
-
-    @Override
-    public BagState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public Iterable<T> read() {
-      try {
-        Iterable<T> result = flinkStateBackend.getOperatorState(descriptor).get();
-        return result != null ? result : Collections.<T>emptyList();
-      } catch (Exception e) {
-        throw new RuntimeException("Error updating state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            Iterable<T> result = flinkStateBackend.getOperatorState(descriptor).get();
-            // PartitionableListState.get() return empty collection When there is no element,
-            // KeyedListState different. (return null)
-            return result == null || Iterators.size(result.iterator()) == 0;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getOperatorState(descriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkSplitBagState<?, ?> that = (FlinkSplitBagState<?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 4f961e5..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ /dev/null
@@ -1,1053 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import com.google.common.collect.Lists;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-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.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.CoderUtils;
-import org.apache.beam.sdk.util.CombineContextFactory;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MapState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.SetState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.joda.time.Instant;
-
-/**
- * {@link StateInternals} that uses a Flink {@link KeyedStateBackend} to manage state.
- *
- * <p>Note: In the Flink streaming runner the key is always encoded
- * using an {@link Coder} and stored in a {@link ByteBuffer}.
- */
-public class FlinkStateInternals<K> implements StateInternals<K> {
-
-  private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-  private Coder<K> keyCoder;
-
-  // on recovery, these will no be properly set because we don't
-  // know which watermark hold states there are in the Flink State Backend
-  private final Map<String, Instant> watermarkHolds = new HashMap<>();
-
-  public FlinkStateInternals(KeyedStateBackend<ByteBuffer> flinkStateBackend, Coder<K> keyCoder) {
-    this.flinkStateBackend = flinkStateBackend;
-    this.keyCoder = keyCoder;
-  }
-
-  /**
-   * Returns the minimum over all watermark holds.
-   */
-  public Instant watermarkHold() {
-    long min = Long.MAX_VALUE;
-    for (Instant hold: watermarkHolds.values()) {
-      min = Math.min(min, hold.getMillis());
-    }
-    return new Instant(min);
-  }
-
-  @Override
-  public K getKey() {
-    ByteBuffer keyBytes = flinkStateBackend.getCurrentKey();
-    try {
-      return CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array());
-    } catch (CoderException e) {
-      throw new RuntimeException("Error decoding key.", e);
-    }
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address) {
-
-    return state(namespace, address, StateContexts.nullContext());
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address,
-      final StateContext<?> context) {
-
-    return address.bind(new StateTag.StateBinder<K>() {
-
-      @Override
-      public <T> ValueState<T> bindValue(
-          StateTag<? super K, ValueState<T>> address,
-          Coder<T> coder) {
-
-        return new FlinkValueState<>(flinkStateBackend, address, namespace, coder);
-      }
-
-      @Override
-      public <T> BagState<T> bindBag(
-          StateTag<? super K, BagState<T>> address,
-          Coder<T> elemCoder) {
-
-        return new FlinkBagState<>(flinkStateBackend, address, namespace, elemCoder);
-      }
-
-      @Override
-      public <T> SetState<T> bindSet(
-          StateTag<? super K, SetState<T>> address,
-          Coder<T> elemCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", SetState.class.getSimpleName()));
-      }
-
-      @Override
-      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-          StateTag<? super K, MapState<KeyT, ValueT>> spec,
-          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", MapState.class.getSimpleName()));
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT>
-      bindCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
-
-        return new FlinkCombiningState<>(
-            flinkStateBackend, address, combineFn, namespace, accumCoder);
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
-        return new FlinkKeyedCombiningState<>(
-            flinkStateBackend,
-            address,
-            combineFn,
-            namespace,
-            accumCoder,
-            FlinkStateInternals.this);
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          CombineWithContext.KeyedCombineFnWithContext<
-              ? super K, InputT, AccumT, OutputT> combineFn) {
-        return new FlinkCombiningStateWithContext<>(
-            flinkStateBackend,
-            address,
-            combineFn,
-            namespace,
-            accumCoder,
-            FlinkStateInternals.this,
-            CombineContextFactory.createFromStateContext(context));
-      }
-
-      @Override
-      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
-          StateTag<? super K, WatermarkHoldState<W>> address,
-          OutputTimeFn<? super W> outputTimeFn) {
-
-        return new FlinkWatermarkHoldState<>(
-            flinkStateBackend, FlinkStateInternals.this, address, namespace, outputTimeFn);
-      }
-    });
-  }
-
-  private static class FlinkValueState<K, T> implements ValueState<T> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, ValueState<T>> address;
-    private final ValueStateDescriptor<T> flinkStateDescriptor;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-
-    FlinkValueState(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        StateTag<? super K, ValueState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-
-      this.namespace = namespace;
-      this.address = address;
-      this.flinkStateBackend = flinkStateBackend;
-
-      CoderTypeInformation<T> typeInfo = new CoderTypeInformation<>(coder);
-
-      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
-    }
-
-    @Override
-    public void write(T input) {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).update(input);
-      } catch (Exception e) {
-        throw new RuntimeException("Error updating state.", e);
-      }
-    }
-
-    @Override
-    public ValueState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public T read() {
-      try {
-        return flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).value();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkValueState<?, ?> that = (FlinkValueState<?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private static class FlinkBagState<K, T> implements BagState<T> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, BagState<T>> address;
-    private final ListStateDescriptor<T> flinkStateDescriptor;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-
-    FlinkBagState(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        StateTag<? super K, BagState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-
-      this.namespace = namespace;
-      this.address = address;
-      this.flinkStateBackend = flinkStateBackend;
-
-      CoderTypeInformation<T> typeInfo = new CoderTypeInformation<>(coder);
-
-      flinkStateDescriptor = new ListStateDescriptor<>(address.getId(), typeInfo);
-    }
-
-    @Override
-    public void add(T input) {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).add(input);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to bag state.", e);
-      }
-    }
-
-    @Override
-    public BagState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public Iterable<T> read() {
-      try {
-        Iterable<T> result = flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).get();
-
-        return result != null ? result : Collections.<T>emptyList();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            Iterable<T> result = flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor).get();
-            return result == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkBagState<?, ?> that = (FlinkBagState<?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private static class FlinkCombiningState<K, InputT, AccumT, OutputT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
-    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-
-    FlinkCombiningState(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder) {
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-      this.flinkStateBackend = flinkStateBackend;
-
-      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
-
-      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          current = combineFn.createAccumulator();
-        }
-        current = combineFn.addInput(current, value);
-        state.update(current);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state." , e);
-      }
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-              namespace.stringKey(),
-              StringSerializer.INSTANCE,
-              flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          state.update(accum);
-        } else {
-          current = combineFn.mergeAccumulators(Lists.newArrayList(current, accum));
-          state.update(current);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state.", e);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      try {
-        return flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).value();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(accumulators);
-    }
-
-    @Override
-    public OutputT read() {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT accum = state.value();
-        if (accum != null) {
-          return combineFn.extractOutput(accum);
-        } else {
-          return combineFn.extractOutput(combineFn.createAccumulator());
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor).value() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkCombiningState<?, ?, ?, ?> that =
-          (FlinkCombiningState<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private static class FlinkKeyedCombiningState<K, InputT, AccumT, OutputT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn;
-    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-    private final FlinkStateInternals<K> flinkStateInternals;
-
-    FlinkKeyedCombiningState(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder,
-        FlinkStateInternals<K> flinkStateInternals) {
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-      this.flinkStateBackend = flinkStateBackend;
-      this.flinkStateInternals = flinkStateInternals;
-
-      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
-
-      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          current = combineFn.createAccumulator(flinkStateInternals.getKey());
-        }
-        current = combineFn.addInput(flinkStateInternals.getKey(), current, value);
-        state.update(current);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state." , e);
-      }
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          state.update(accum);
-        } else {
-          current = combineFn.mergeAccumulators(
-              flinkStateInternals.getKey(),
-              Lists.newArrayList(current, accum));
-          state.update(current);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state.", e);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      try {
-        return flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).value();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators);
-    }
-
-    @Override
-    public OutputT read() {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT accum = state.value();
-        if (accum != null) {
-          return combineFn.extractOutput(flinkStateInternals.getKey(), accum);
-        } else {
-          return combineFn.extractOutput(
-              flinkStateInternals.getKey(),
-              combineFn.createAccumulator(flinkStateInternals.getKey()));
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor).value() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkKeyedCombiningState<?, ?, ?, ?> that =
-          (FlinkKeyedCombiningState<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private static class FlinkCombiningStateWithContext<K, InputT, AccumT, OutputT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final CombineWithContext.KeyedCombineFnWithContext<
-        ? super K, InputT, AccumT, OutputT> combineFn;
-    private final ValueStateDescriptor<AccumT> flinkStateDescriptor;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-    private final FlinkStateInternals<K> flinkStateInternals;
-    private final CombineWithContext.Context context;
-
-    FlinkCombiningStateWithContext(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        CombineWithContext.KeyedCombineFnWithContext<
-            ? super K, InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder,
-        FlinkStateInternals<K> flinkStateInternals,
-        CombineWithContext.Context context) {
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-      this.flinkStateBackend = flinkStateBackend;
-      this.flinkStateInternals = flinkStateInternals;
-      this.context = context;
-
-      CoderTypeInformation<AccumT> typeInfo = new CoderTypeInformation<>(accumCoder);
-
-      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          current = combineFn.createAccumulator(flinkStateInternals.getKey(), context);
-        }
-        current = combineFn.addInput(flinkStateInternals.getKey(), current, value, context);
-        state.update(current);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state." , e);
-      }
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT current = state.value();
-        if (current == null) {
-          state.update(accum);
-        } else {
-          current = combineFn.mergeAccumulators(
-              flinkStateInternals.getKey(),
-              Lists.newArrayList(current, accum),
-              context);
-          state.update(current);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state.", e);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      try {
-        return flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).value();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators, context);
-    }
-
-    @Override
-    public OutputT read() {
-      try {
-        org.apache.flink.api.common.state.ValueState<AccumT> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-
-        AccumT accum = state.value();
-        return combineFn.extractOutput(flinkStateInternals.getKey(), accum, context);
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor).value() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      try {
-        flinkStateBackend.getPartitionedState(
-            namespace.stringKey(),
-            StringSerializer.INSTANCE,
-            flinkStateDescriptor).clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkCombiningStateWithContext<?, ?, ?, ?> that =
-          (FlinkCombiningStateWithContext<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private static class FlinkWatermarkHoldState<K, W extends BoundedWindow>
-      implements WatermarkHoldState<W> {
-    private final StateTag<? super K, WatermarkHoldState<W>> address;
-    private final OutputTimeFn<? super W> outputTimeFn;
-    private final StateNamespace namespace;
-    private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-    private final FlinkStateInternals<K> flinkStateInternals;
-    private final ValueStateDescriptor<Instant> flinkStateDescriptor;
-
-    public FlinkWatermarkHoldState(
-        KeyedStateBackend<ByteBuffer> flinkStateBackend,
-        FlinkStateInternals<K> flinkStateInternals,
-        StateTag<? super K, WatermarkHoldState<W>> address,
-        StateNamespace namespace,
-        OutputTimeFn<? super W> outputTimeFn) {
-      this.address = address;
-      this.outputTimeFn = outputTimeFn;
-      this.namespace = namespace;
-      this.flinkStateBackend = flinkStateBackend;
-      this.flinkStateInternals = flinkStateInternals;
-
-      CoderTypeInformation<Instant> typeInfo = new CoderTypeInformation<>(InstantCoder.of());
-      flinkStateDescriptor = new ValueStateDescriptor<>(address.getId(), typeInfo, null);
-    }
-
-    @Override
-    public OutputTimeFn<? super W> getOutputTimeFn() {
-      return outputTimeFn;
-    }
-
-    @Override
-    public WatermarkHoldState<W> readLater() {
-      return this;
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor).value() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-
-    }
-
-    @Override
-    public void add(Instant value) {
-      try {
-        org.apache.flink.api.common.state.ValueState<Instant> state =
-            flinkStateBackend.getPartitionedState(
-              namespace.stringKey(),
-              StringSerializer.INSTANCE,
-              flinkStateDescriptor);
-
-        Instant current = state.value();
-        if (current == null) {
-          state.update(value);
-          flinkStateInternals.watermarkHolds.put(namespace.stringKey(), value);
-        } else {
-          Instant combined = outputTimeFn.combine(current, value);
-          state.update(combined);
-          flinkStateInternals.watermarkHolds.put(namespace.stringKey(), combined);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error updating state.", e);
-      }
-    }
-
-    @Override
-    public Instant read() {
-      try {
-        org.apache.flink.api.common.state.ValueState<Instant> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-        return state.value();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public void clear() {
-      flinkStateInternals.watermarkHolds.remove(namespace.stringKey());
-      try {
-        org.apache.flink.api.common.state.ValueState<Instant> state =
-            flinkStateBackend.getPartitionedState(
-                namespace.stringKey(),
-                StringSerializer.INSTANCE,
-                flinkStateDescriptor);
-        state.clear();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkWatermarkHoldState<?, ?> that = (FlinkWatermarkHoldState<?, ?>) o;
-
-      if (!address.equals(that.address)) {
-        return false;
-      }
-      if (!outputTimeFn.equals(that.outputTimeFn)) {
-        return false;
-      }
-      return namespace.equals(that.namespace);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = address.hashCode();
-      result = 31 * result + outputTimeFn.hashCode();
-      result = 31 * result + namespace.hashCode();
-      return result;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.java
deleted file mode 100644
index b38a520..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupCheckpointedOperator.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 org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import java.io.DataOutputStream;
-
-/**
- * This interface is used to checkpoint key-groups state.
- */
-public interface KeyGroupCheckpointedOperator extends KeyGroupRestoringOperator{
-  /**
-   * Snapshots the state for a given {@code keyGroupIdx}.
-   *
-   * <p>AbstractStreamOperator would call this hook in
-   * AbstractStreamOperator.snapshotState() while iterating over the key groups.
-   * @param keyGroupIndex the id of the key-group to be put in the snapshot.
-   * @param out the stream to write to.
-   */
-  void snapshotKeyGroupState(int keyGroupIndex, DataOutputStream out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.java
deleted file mode 100644
index 2bdfc6e..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/KeyGroupRestoringOperator.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 org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import java.io.DataInputStream;
-
-/**
- * This interface is used to restore key-groups state.
- */
-public interface KeyGroupRestoringOperator {
-  /**
-   * Restore the state for a given {@code keyGroupIndex}.
-   * @param keyGroupIndex the id of the key-group to be put in the snapshot.
-   * @param in the stream to read from.
-   */
-  void restoreKeyGroupState(int keyGroupIndex, DataInputStream in) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
deleted file mode 100644
index 0004e9e..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal state implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.state;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/resources/log4j.properties b/runners/flink/runner/src/main/resources/log4j.properties
deleted file mode 100644
index 4b6a708..0000000
--- a/runners/flink/runner/src/main/resources/log4j.properties
+++ /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.
-################################################################################
-
-log4j.rootLogger=OFF,console
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
deleted file mode 100644
index 10d6d9d..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.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 org.apache.beam.runners.flink;
-
-import org.apache.beam.runners.flink.translation.types.EncodedValueComparator;
-import org.apache.beam.runners.flink.translation.types.EncodedValueTypeInformation;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.ComparatorTestBase;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.junit.Assert;
-
-/**
- * Test for {@link EncodedValueComparator}.
- */
-public class EncodedValueComparatorTest extends ComparatorTestBase<byte[]> {
-
-  @Override
-  protected TypeComparator<byte[]> createComparator(boolean ascending) {
-    return new EncodedValueTypeInformation().createComparator(ascending, new ExecutionConfig());
-  }
-
-  @Override
-  protected TypeSerializer<byte[]> createSerializer() {
-    return new EncodedValueTypeInformation().createSerializer(new ExecutionConfig());
-  }
-
-  @Override
-  protected void deepEquals(String message, byte[] should, byte[] is) {
-    Assert.assertArrayEquals(message, should, is);
-  }
-
-  @Override
-  protected byte[][] getSortedTestData() {
-    StringUtf8Coder coder = StringUtf8Coder.of();
-
-    try {
-      return new byte[][]{
-          CoderUtils.encodeToByteArray(coder, ""),
-          CoderUtils.encodeToByteArray(coder, "Lorem Ipsum Dolor Omit Longer"),
-          CoderUtils.encodeToByteArray(coder, "aaaa"),
-          CoderUtils.encodeToByteArray(coder, "abcd"),
-          CoderUtils.encodeToByteArray(coder, "abce"),
-          CoderUtils.encodeToByteArray(coder, "abdd"),
-          CoderUtils.encodeToByteArray(coder, "accd"),
-          CoderUtils.encodeToByteArray(coder, "bbcd")
-      };
-    } catch (CoderException e) {
-      throw new RuntimeException("Could not encode values.", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index d9d174c..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.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 org.apache.beam.runners.flink;
-
-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.
- */
-public class FlinkRunnerRegistrarTest {
-
-  @Test
-  public void testFullName() {
-    String[] args =
-        new String[] {String.format("--runner=%s", FlinkRunner.class.getName())};
-    PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
-    assertEquals(opts.getRunner(), FlinkRunner.class);
-  }
-
-  @Test
-  public void testClassName() {
-    String[] args =
-        new String[] {String.format("--runner=%s", FlinkRunner.class.getSimpleName())};
-    PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
-    assertEquals(opts.getRunner(), FlinkRunner.class);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index d6240c4..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-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 org.apache.beam.sdk.Pipeline} for testing Dataflow programs on the
- * {@link FlinkRunner}.
- */
-public class FlinkTestPipeline extends Pipeline {
-
-  /**
-   * Creates and returns a new test pipeline for batch execution.
-   *
-   * <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() {
-    return create(false);
-  }
-
-  /**
-   * Creates and returns a new test pipeline for streaming execution.
-   *
-   * <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
-   */
-  public static FlinkTestPipeline createForStreaming() {
-    return create(true);
-  }
-
-  /**
-   * Creates and returns a new test pipeline for streaming or batch execution.
-   *
-   * <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.
-   * @return The Test Pipeline.
-   */
-  private static FlinkTestPipeline create(boolean streaming) {
-    TestFlinkRunner flinkRunner = TestFlinkRunner.create(streaming);
-    return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
-  }
-
-  private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
-              PipelineOptions options) {
-    super(runner, options);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
deleted file mode 100644
index 06187f6..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.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 org.apache.beam.runners.flink;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-import java.util.HashMap;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-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.DoFn;
-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.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.joda.time.Instant;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Tests for serialization and deserialization of {@link PipelineOptions} in {@link DoFnOperator}.
- */
-public class PipelineOptionsTest {
-
-  /**
-   * Pipeline options.
-   */
-  public interface MyOptions extends FlinkPipelineOptions {
-    @Description("Bla bla bla")
-    @Default.String("Hello")
-    String getTestOption();
-    void setTestOption(String value);
-  }
-
-  private static MyOptions options;
-  private static SerializedPipelineOptions serializedOptions;
-
-  private static final String[] args = new String[]{"--testOption=nothing"};
-
-  @BeforeClass
-  public static void beforeTest() {
-    options = PipelineOptionsFactory.fromArgs(args).as(MyOptions.class);
-    serializedOptions = new SerializedPipelineOptions(options);
-  }
-
-  @Test
-  public void testDeserialization() {
-    MyOptions deserializedOptions = serializedOptions.getPipelineOptions().as(MyOptions.class);
-    assertEquals("nothing", deserializedOptions.getTestOption());
-  }
-
-  @Test
-  public void testIgnoredFieldSerialization() {
-    FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
-    options.setStateBackend(new MemoryStateBackend());
-
-    FlinkPipelineOptions deserialized =
-        new SerializedPipelineOptions(options).getPipelineOptions().as(FlinkPipelineOptions.class);
-
-    assertNull(deserialized.getStateBackend());
-  }
-
-  @Test
-  public void testCaching() {
-    PipelineOptions deserializedOptions =
-        serializedOptions.getPipelineOptions().as(PipelineOptions.class);
-
-    assertNotNull(deserializedOptions);
-    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
-    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
-    assertTrue(deserializedOptions == serializedOptions.getPipelineOptions());
-  }
-
-  @Test(expected = Exception.class)
-  public void testNonNull() {
-    new SerializedPipelineOptions(null);
-  }
-
-  @Test(expected = Exception.class)
-  public void parDoBaseClassPipelineOptionsNullTest() {
-    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
-        new TestDoFn(),
-        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()),
-        new TupleTag<String>("main-output"),
-        Collections.<TupleTag<?>>emptyList(),
-        new DoFnOperator.DefaultOutputManagerFactory<String>(),
-        WindowingStrategy.globalDefault(),
-        new HashMap<Integer, PCollectionView<?>>(),
-        Collections.<PCollectionView<?>>emptyList(),
-        null,
-        null);
-
-  }
-
-  /**
-   * Tests that PipelineOptions are present after serialization.
-   */
-  @Test
-  public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception {
-
-    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
-        new TestDoFn(),
-        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()),
-        new TupleTag<String>("main-output"),
-        Collections.<TupleTag<?>>emptyList(),
-        new DoFnOperator.DefaultOutputManagerFactory<String>(),
-        WindowingStrategy.globalDefault(),
-        new HashMap<Integer, PCollectionView<?>>(),
-        Collections.<PCollectionView<?>>emptyList(),
-        options,
-        null);
-
-    final byte[] serialized = SerializationUtils.serialize(doFnOperator);
-
-    @SuppressWarnings("unchecked")
-    DoFnOperator<Object, Object, Object> deserialized =
-        (DoFnOperator<Object, Object, Object>) SerializationUtils.deserialize(serialized);
-
-    TypeInformation<WindowedValue<Object>> typeInformation = TypeInformation.of(
-        new TypeHint<WindowedValue<Object>>() {});
-
-    OneInputStreamOperatorTestHarness<WindowedValue<Object>, Object> testHarness =
-        new OneInputStreamOperatorTestHarness<>(deserialized,
-            typeInformation.createSerializer(new ExecutionConfig()));
-
-    testHarness.open();
-
-    // execute once to access options
-    testHarness.processElement(new StreamRecord<>(
-        WindowedValue.of(
-            new Object(),
-            Instant.now(),
-            GlobalWindow.INSTANCE,
-            PaneInfo.NO_FIRING)));
-
-    testHarness.close();
-
-  }
-
-
-  private static class TestDoFn extends DoFn<String, String> {
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      Assert.assertNotNull(c.getPipelineOptions());
-      Assert.assertEquals(
-          options.getTestOption(),
-          c.getPipelineOptions().as(MyOptions.class).getTestOption());
-    }
-  }
-}


[18/18] beam git commit: This closes #2576

Posted by ie...@apache.org.
This closes #2576


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

Branch: refs/heads/master
Commit: 83193698d8ea3dc9cb2a3ed8fe6b4ee6b810237c
Parents: 8a00f22 cdd2544
Author: Isma�l Mej�a <ie...@apache.org>
Authored: Wed Apr 19 15:07:54 2017 +0200
Committer: Isma�l Mej�a <ie...@apache.org>
Committed: Wed Apr 19 15:07:54 2017 +0200

----------------------------------------------------------------------
 ...PostCommit_Java_ValidatesRunner_Flink.groovy |    2 +-
 runners/flink/examples/pom.xml                  |  130 ---
 .../beam/runners/flink/examples/TFIDF.java      |  455 --------
 .../beam/runners/flink/examples/WordCount.java  |  129 ---
 .../runners/flink/examples/package-info.java    |   22 -
 .../flink/examples/streaming/AutoComplete.java  |  400 -------
 .../flink/examples/streaming/JoinExamples.java  |  154 ---
 .../examples/streaming/WindowedWordCount.java   |  141 ---
 .../flink/examples/streaming/package-info.java  |   22 -
 runners/flink/pom.xml                           |  275 ++++-
 runners/flink/runner/pom.xml                    |  330 ------
 .../flink/DefaultParallelismFactory.java        |   39 -
 .../flink/FlinkBatchPipelineTranslator.java     |  139 ---
 .../flink/FlinkBatchTransformTranslators.java   |  723 ------------
 .../flink/FlinkBatchTranslationContext.java     |  153 ---
 .../flink/FlinkDetachedRunnerResult.java        |   75 --
 .../FlinkPipelineExecutionEnvironment.java      |  241 ----
 .../runners/flink/FlinkPipelineOptions.java     |  101 --
 .../runners/flink/FlinkPipelineTranslator.java  |   53 -
 .../apache/beam/runners/flink/FlinkRunner.java  |  232 ----
 .../runners/flink/FlinkRunnerRegistrar.java     |   62 --
 .../beam/runners/flink/FlinkRunnerResult.java   |   98 --
 .../flink/FlinkStreamingPipelineTranslator.java |  276 -----
 .../FlinkStreamingTransformTranslators.java     | 1044 -----------------
 .../flink/FlinkStreamingTranslationContext.java |  130 ---
 .../flink/FlinkStreamingViewOverrides.java      |  372 -------
 .../flink/PipelineTranslationOptimizer.java     |   72 --
 .../beam/runners/flink/TestFlinkRunner.java     |   84 --
 .../beam/runners/flink/TranslationMode.java     |   31 -
 .../apache/beam/runners/flink/package-info.java |   22 -
 .../functions/FlinkAggregatorFactory.java       |   53 -
 .../functions/FlinkAssignContext.java           |   63 --
 .../functions/FlinkAssignWindows.java           |   49 -
 .../functions/FlinkDoFnFunction.java            |  161 ---
 .../FlinkMergingNonShuffleReduceFunction.java   |  228 ----
 .../FlinkMergingPartialReduceFunction.java      |  201 ----
 .../functions/FlinkMergingReduceFunction.java   |  199 ----
 .../FlinkMultiOutputPruningFunction.java        |   50 -
 .../functions/FlinkNoOpStepContext.java         |   73 --
 .../functions/FlinkPartialReduceFunction.java   |  172 ---
 .../functions/FlinkReduceFunction.java          |  173 ---
 .../functions/FlinkSideInputReader.java         |   80 --
 .../functions/FlinkStatefulDoFnFunction.java    |  198 ----
 .../functions/SideInputInitializer.java         |   73 --
 .../translation/functions/package-info.java     |   22 -
 .../runners/flink/translation/package-info.java |   22 -
 .../translation/types/CoderTypeInformation.java |  120 --
 .../translation/types/CoderTypeSerializer.java  |  132 ---
 .../types/EncodedValueComparator.java           |  195 ----
 .../types/EncodedValueSerializer.java           |  113 --
 .../types/EncodedValueTypeInformation.java      |   98 --
 .../types/InspectableByteArrayOutputStream.java |   34 -
 .../flink/translation/types/KvKeySelector.java  |   50 -
 .../flink/translation/types/package-info.java   |   22 -
 .../utils/SerializedPipelineOptions.java        |   67 --
 .../flink/translation/utils/package-info.java   |   22 -
 .../wrappers/DataInputViewWrapper.java          |   58 -
 .../wrappers/DataOutputViewWrapper.java         |   51 -
 .../SerializableFnAggregatorWrapper.java        |   98 --
 .../translation/wrappers/SourceInputFormat.java |  150 ---
 .../translation/wrappers/SourceInputSplit.java  |   52 -
 .../translation/wrappers/package-info.java      |   22 -
 .../wrappers/streaming/DoFnOperator.java        |  774 -------------
 .../streaming/KvToByteBufferKeySelector.java    |   56 -
 .../streaming/SingletonKeyedWorkItem.java       |   56 -
 .../streaming/SingletonKeyedWorkItemCoder.java  |  126 ---
 .../streaming/SplittableDoFnOperator.java       |  150 ---
 .../wrappers/streaming/WindowDoFnOperator.java  |  117 --
 .../wrappers/streaming/WorkItemKeySelector.java |   56 -
 .../streaming/io/BoundedSourceWrapper.java      |  218 ----
 .../streaming/io/UnboundedSocketSource.java     |  249 -----
 .../streaming/io/UnboundedSourceWrapper.java    |  476 --------
 .../wrappers/streaming/io/package-info.java     |   22 -
 .../wrappers/streaming/package-info.java        |   22 -
 .../state/FlinkBroadcastStateInternals.java     |  865 --------------
 .../state/FlinkKeyGroupStateInternals.java      |  487 --------
 .../state/FlinkSplitStateInternals.java         |  260 -----
 .../streaming/state/FlinkStateInternals.java    | 1053 ------------------
 .../state/KeyGroupCheckpointedOperator.java     |   35 -
 .../state/KeyGroupRestoringOperator.java        |   32 -
 .../wrappers/streaming/state/package-info.java  |   22 -
 .../runner/src/main/resources/log4j.properties  |   23 -
 .../flink/EncodedValueComparatorTest.java       |   70 --
 .../runners/flink/FlinkRunnerRegistrarTest.java |   48 -
 .../beam/runners/flink/FlinkTestPipeline.java   |   72 --
 .../beam/runners/flink/PipelineOptionsTest.java |  184 ---
 .../beam/runners/flink/ReadSourceITCase.java    |   85 --
 .../flink/ReadSourceStreamingITCase.java        |   74 --
 .../beam/runners/flink/WriteSinkITCase.java     |  192 ----
 .../flink/streaming/DoFnOperatorTest.java       |  600 ----------
 .../FlinkBroadcastStateInternalsTest.java       |  245 ----
 .../FlinkKeyGroupStateInternalsTest.java        |  262 -----
 .../streaming/FlinkSplitStateInternalsTest.java |  101 --
 .../streaming/FlinkStateInternalsTest.java      |  395 -------
 .../flink/streaming/GroupByNullKeyTest.java     |  124 ---
 .../flink/streaming/TestCountingSource.java     |  254 -----
 .../streaming/TopWikipediaSessionsITCase.java   |  133 ---
 .../streaming/UnboundedSourceWrapperTest.java   |  464 --------
 .../runners/flink/streaming/package-info.java   |   22 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../flink/DefaultParallelismFactory.java        |   39 +
 .../flink/FlinkBatchPipelineTranslator.java     |  139 +++
 .../flink/FlinkBatchTransformTranslators.java   |  723 ++++++++++++
 .../flink/FlinkBatchTranslationContext.java     |  153 +++
 .../flink/FlinkDetachedRunnerResult.java        |   75 ++
 .../FlinkPipelineExecutionEnvironment.java      |  241 ++++
 .../runners/flink/FlinkPipelineOptions.java     |  101 ++
 .../runners/flink/FlinkPipelineTranslator.java  |   53 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  232 ++++
 .../runners/flink/FlinkRunnerRegistrar.java     |   62 ++
 .../beam/runners/flink/FlinkRunnerResult.java   |   98 ++
 .../flink/FlinkStreamingPipelineTranslator.java |  276 +++++
 .../FlinkStreamingTransformTranslators.java     | 1044 +++++++++++++++++
 .../flink/FlinkStreamingTranslationContext.java |  130 +++
 .../flink/FlinkStreamingViewOverrides.java      |  372 +++++++
 .../flink/PipelineTranslationOptimizer.java     |   72 ++
 .../beam/runners/flink/TestFlinkRunner.java     |   84 ++
 .../beam/runners/flink/TranslationMode.java     |   31 +
 .../apache/beam/runners/flink/package-info.java |   22 +
 .../functions/FlinkAggregatorFactory.java       |   53 +
 .../functions/FlinkAssignContext.java           |   63 ++
 .../functions/FlinkAssignWindows.java           |   49 +
 .../functions/FlinkDoFnFunction.java            |  161 +++
 .../FlinkMergingNonShuffleReduceFunction.java   |  228 ++++
 .../FlinkMergingPartialReduceFunction.java      |  201 ++++
 .../functions/FlinkMergingReduceFunction.java   |  199 ++++
 .../FlinkMultiOutputPruningFunction.java        |   50 +
 .../functions/FlinkNoOpStepContext.java         |   73 ++
 .../functions/FlinkPartialReduceFunction.java   |  172 +++
 .../functions/FlinkReduceFunction.java          |  173 +++
 .../functions/FlinkSideInputReader.java         |   80 ++
 .../functions/FlinkStatefulDoFnFunction.java    |  198 ++++
 .../functions/SideInputInitializer.java         |   73 ++
 .../translation/functions/package-info.java     |   22 +
 .../runners/flink/translation/package-info.java |   22 +
 .../translation/types/CoderTypeInformation.java |  120 ++
 .../translation/types/CoderTypeSerializer.java  |  132 +++
 .../types/EncodedValueComparator.java           |  195 ++++
 .../types/EncodedValueSerializer.java           |  113 ++
 .../types/EncodedValueTypeInformation.java      |   98 ++
 .../types/InspectableByteArrayOutputStream.java |   34 +
 .../flink/translation/types/KvKeySelector.java  |   50 +
 .../flink/translation/types/package-info.java   |   22 +
 .../utils/SerializedPipelineOptions.java        |   67 ++
 .../flink/translation/utils/package-info.java   |   22 +
 .../wrappers/DataInputViewWrapper.java          |   58 +
 .../wrappers/DataOutputViewWrapper.java         |   51 +
 .../SerializableFnAggregatorWrapper.java        |   98 ++
 .../translation/wrappers/SourceInputFormat.java |  150 +++
 .../translation/wrappers/SourceInputSplit.java  |   52 +
 .../translation/wrappers/package-info.java      |   22 +
 .../wrappers/streaming/DoFnOperator.java        |  774 +++++++++++++
 .../streaming/KvToByteBufferKeySelector.java    |   56 +
 .../streaming/SingletonKeyedWorkItem.java       |   56 +
 .../streaming/SingletonKeyedWorkItemCoder.java  |  126 +++
 .../streaming/SplittableDoFnOperator.java       |  150 +++
 .../wrappers/streaming/WindowDoFnOperator.java  |  117 ++
 .../wrappers/streaming/WorkItemKeySelector.java |   56 +
 .../streaming/io/BoundedSourceWrapper.java      |  218 ++++
 .../streaming/io/UnboundedSocketSource.java     |  249 +++++
 .../streaming/io/UnboundedSourceWrapper.java    |  476 ++++++++
 .../wrappers/streaming/io/package-info.java     |   22 +
 .../wrappers/streaming/package-info.java        |   22 +
 .../state/FlinkBroadcastStateInternals.java     |  865 ++++++++++++++
 .../state/FlinkKeyGroupStateInternals.java      |  487 ++++++++
 .../state/FlinkSplitStateInternals.java         |  260 +++++
 .../streaming/state/FlinkStateInternals.java    | 1053 ++++++++++++++++++
 .../state/KeyGroupCheckpointedOperator.java     |   35 +
 .../state/KeyGroupRestoringOperator.java        |   32 +
 .../wrappers/streaming/state/package-info.java  |   22 +
 .../flink/src/main/resources/log4j.properties   |   23 +
 .../flink/EncodedValueComparatorTest.java       |   70 ++
 .../runners/flink/FlinkRunnerRegistrarTest.java |   48 +
 .../beam/runners/flink/FlinkTestPipeline.java   |   72 ++
 .../beam/runners/flink/PipelineOptionsTest.java |  184 +++
 .../beam/runners/flink/ReadSourceITCase.java    |   85 ++
 .../flink/ReadSourceStreamingITCase.java        |   74 ++
 .../beam/runners/flink/WriteSinkITCase.java     |  192 ++++
 .../flink/streaming/DoFnOperatorTest.java       |  600 ++++++++++
 .../FlinkBroadcastStateInternalsTest.java       |  245 ++++
 .../FlinkKeyGroupStateInternalsTest.java        |  262 +++++
 .../streaming/FlinkSplitStateInternalsTest.java |  101 ++
 .../streaming/FlinkStateInternalsTest.java      |  395 +++++++
 .../flink/streaming/GroupByNullKeyTest.java     |  124 +++
 .../flink/streaming/TestCountingSource.java     |  254 +++++
 .../streaming/TopWikipediaSessionsITCase.java   |  133 +++
 .../streaming/UnboundedSourceWrapperTest.java   |  464 ++++++++
 .../runners/flink/streaming/package-info.java   |   22 +
 .../src/test/resources/log4j-test.properties    |   27 +
 189 files changed, 15765 insertions(+), 17293 deletions(-)
----------------------------------------------------------------------



[06/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
new file mode 100644
index 0000000..26fd0b4
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.PerKeyCombineFnRunner;
+import org.apache.beam.runners.core.PerKeyCombineFnRunners;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.PaneInfo;
+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.PCollectionView;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * Special version of {@link FlinkReduceFunction} that supports merging windows. This
+ * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
+ * same behaviour as {@code MergeOverlappingIntervalWindows}.
+ *
+ * <p>This is different from the pair of function for the non-merging windows case
+ * in that we cannot do combining before the shuffle because elements would not
+ * yet be in their correct windows for side-input access.
+ */
+public class FlinkMergingNonShuffleReduceFunction<
+    K, InputT, AccumT, OutputT, W extends IntervalWindow>
+    extends RichGroupReduceFunction<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, OutputT>>> {
+
+  private final CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> combineFn;
+
+  private final WindowingStrategy<?, W> windowingStrategy;
+
+  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
+
+  private final SerializedPipelineOptions serializedOptions;
+
+  public FlinkMergingNonShuffleReduceFunction(
+      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> keyedCombineFn,
+      WindowingStrategy<?, W> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions) {
+
+    this.combineFn = keyedCombineFn;
+
+    this.windowingStrategy = windowingStrategy;
+    this.sideInputs = sideInputs;
+
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+
+  }
+
+  @Override
+  public void reduce(
+      Iterable<WindowedValue<KV<K, InputT>>> elements,
+      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
+
+    PipelineOptions options = serializedOptions.getPipelineOptions();
+
+    FlinkSideInputReader sideInputReader =
+        new FlinkSideInputReader(sideInputs, getRuntimeContext());
+
+    PerKeyCombineFnRunner<K, InputT, AccumT, OutputT> combineFnRunner =
+        PerKeyCombineFnRunners.create(combineFn);
+
+    @SuppressWarnings("unchecked")
+    OutputTimeFn<? super BoundedWindow> outputTimeFn =
+        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
+
+    // get all elements so that we can sort them, has to fit into
+    // memory
+    // this seems very unprudent, but correct, for now
+    List<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
+    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
+      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
+        sortedInput.add(exploded);
+      }
+    }
+    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
+      @Override
+      public int compare(
+          WindowedValue<KV<K, InputT>> o1,
+          WindowedValue<KV<K, InputT>> o2) {
+        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
+            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
+      }
+    });
+
+    // merge windows, we have to do it in an extra pre-processing step and
+    // can't do it as we go since the window of early elements would not
+    // be correct when calling the CombineFn
+    mergeWindow(sortedInput);
+
+    // iterate over the elements that are sorted by window timestamp
+    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
+
+    // create accumulator using the first elements key
+    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
+    K key = currentValue.getValue().getKey();
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
+    InputT firstValue = currentValue.getValue().getValue();
+    AccumT accumulator =
+        combineFnRunner.createAccumulator(key, options, sideInputReader, currentValue.getWindows());
+    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
+        options, sideInputReader, currentValue.getWindows());
+
+    // we use this to keep track of the timestamps assigned by the OutputTimeFn
+    Instant windowTimestamp =
+        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
+
+    while (iterator.hasNext()) {
+      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
+      IntervalWindow nextWindow =
+          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+
+      if (currentWindow.equals(nextWindow)) {
+        // continue accumulating and merge windows
+
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+
+        windowTimestamp = outputTimeFn.combine(
+            windowTimestamp,
+            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
+
+      } else {
+        // emit the value that we currently have
+        out.collect(
+            WindowedValue.of(
+                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                    options, sideInputReader, currentValue.getWindows())),
+                windowTimestamp,
+                currentWindow,
+                PaneInfo.NO_FIRING));
+
+        currentWindow = nextWindow;
+        currentValue = nextValue;
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.createAccumulator(key,
+            options, sideInputReader, currentValue.getWindows());
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
+      }
+
+    }
+
+    // emit the final accumulator
+    out.collect(
+        WindowedValue.of(
+            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                options, sideInputReader, currentValue.getWindows())),
+            windowTimestamp,
+            currentWindow,
+            PaneInfo.NO_FIRING));
+  }
+
+  /**
+   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
+   * This replaces windows in the input list.
+   */
+  private void mergeWindow(List<WindowedValue<KV<K, InputT>>> elements) {
+    int currentStart = 0;
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
+
+    for (int i = 1; i < elements.size(); i++) {
+      WindowedValue<KV<K, InputT>> nextValue = elements.get(i);
+      IntervalWindow nextWindow =
+          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+      if (currentWindow.intersects(nextWindow)) {
+        // we continue
+        currentWindow = currentWindow.span(nextWindow);
+      } else {
+        // retrofit the merged window to all windows up to "currentStart"
+        for (int j = i - 1; j >= currentStart; j--) {
+          WindowedValue<KV<K, InputT>> value = elements.get(j);
+          elements.set(
+              j,
+              WindowedValue.of(
+                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+        }
+        currentStart = i;
+        currentWindow = nextWindow;
+      }
+    }
+    if (currentStart < elements.size() - 1) {
+      // we have to retrofit the last batch
+      for (int j = elements.size() - 1; j >= currentStart; j--) {
+        WindowedValue<KV<K, InputT>> value = elements.get(j);
+        elements.set(
+            j,
+            WindowedValue.of(
+                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
new file mode 100644
index 0000000..c68f155
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.PerKeyCombineFnRunner;
+import org.apache.beam.runners.core.PerKeyCombineFnRunners;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.PaneInfo;
+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.PCollectionView;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * Special version of {@link FlinkPartialReduceFunction} that supports merging windows. This
+ * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
+ * same behaviour as {@code MergeOverlappingIntervalWindows}.
+ */
+public class FlinkMergingPartialReduceFunction<K, InputT, AccumT, W extends IntervalWindow>
+    extends FlinkPartialReduceFunction<K, InputT, AccumT, W> {
+
+  public FlinkMergingPartialReduceFunction(
+      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn,
+      WindowingStrategy<?, W> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions) {
+    super(combineFn, windowingStrategy, sideInputs, pipelineOptions);
+  }
+
+  @Override
+  public void combine(
+      Iterable<WindowedValue<KV<K, InputT>>> elements,
+      Collector<WindowedValue<KV<K, AccumT>>> out) throws Exception {
+
+    PipelineOptions options = serializedOptions.getPipelineOptions();
+
+    FlinkSideInputReader sideInputReader =
+        new FlinkSideInputReader(sideInputs, getRuntimeContext());
+
+    PerKeyCombineFnRunner<K, InputT, AccumT, ?> combineFnRunner =
+        PerKeyCombineFnRunners.create(combineFn);
+
+    @SuppressWarnings("unchecked")
+    OutputTimeFn<? super BoundedWindow> outputTimeFn =
+        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
+
+    // get all elements so that we can sort them, has to fit into
+    // memory
+    // this seems very unprudent, but correct, for now
+    List<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
+    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
+      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
+        sortedInput.add(exploded);
+      }
+    }
+    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
+      @Override
+      public int compare(
+          WindowedValue<KV<K, InputT>> o1,
+          WindowedValue<KV<K, InputT>> o2) {
+        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
+            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
+      }
+    });
+
+    // merge windows, we have to do it in an extra pre-processing step and
+    // can't do it as we go since the window of early elements would not
+    // be correct when calling the CombineFn
+    mergeWindow(sortedInput);
+
+    // iterate over the elements that are sorted by window timestamp
+    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
+
+    // create accumulator using the first elements key
+    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
+    K key = currentValue.getValue().getKey();
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
+    InputT firstValue = currentValue.getValue().getValue();
+    AccumT accumulator = combineFnRunner.createAccumulator(key,
+        options, sideInputReader, currentValue.getWindows());
+    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
+        options, sideInputReader, currentValue.getWindows());
+
+    // we use this to keep track of the timestamps assigned by the OutputTimeFn
+    Instant windowTimestamp =
+        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
+
+    while (iterator.hasNext()) {
+      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
+      IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+
+      if (currentWindow.equals(nextWindow)) {
+        // continue accumulating and merge windows
+
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+
+        windowTimestamp = outputTimeFn.combine(
+            windowTimestamp,
+            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
+
+      } else {
+        // emit the value that we currently have
+        out.collect(
+            WindowedValue.of(
+                KV.of(key, accumulator),
+                windowTimestamp,
+                currentWindow,
+                PaneInfo.NO_FIRING));
+
+        currentWindow = nextWindow;
+        currentValue = nextValue;
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.createAccumulator(key,
+            options, sideInputReader, currentValue.getWindows());
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
+      }
+    }
+
+    // emit the final accumulator
+    out.collect(
+        WindowedValue.of(
+            KV.of(key, accumulator),
+            windowTimestamp,
+            currentWindow,
+            PaneInfo.NO_FIRING));
+  }
+
+  /**
+   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
+   * This replaces windows in the input list.
+   */
+  private void mergeWindow(List<WindowedValue<KV<K, InputT>>> elements) {
+    int currentStart = 0;
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
+
+    for (int i = 1; i < elements.size(); i++) {
+      WindowedValue<KV<K, InputT>> nextValue = elements.get(i);
+      IntervalWindow nextWindow =
+          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+      if (currentWindow.intersects(nextWindow)) {
+        // we continue
+        currentWindow = currentWindow.span(nextWindow);
+      } else {
+        // retrofit the merged window to all windows up to "currentStart"
+        for (int j = i - 1; j >= currentStart; j--) {
+          WindowedValue<KV<K, InputT>> value = elements.get(j);
+          elements.set(
+              j,
+              WindowedValue.of(
+                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+        }
+        currentStart = i;
+        currentWindow = nextWindow;
+      }
+    }
+    if (currentStart < elements.size() - 1) {
+      // we have to retrofit the last batch
+      for (int j = elements.size() - 1; j >= currentStart; j--) {
+        WindowedValue<KV<K, InputT>> value = elements.get(j);
+        elements.set(
+            j,
+            WindowedValue.of(
+                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java
new file mode 100644
index 0000000..84b3adc
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.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 org.apache.beam.runners.flink.translation.functions;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.PerKeyCombineFnRunner;
+import org.apache.beam.runners.core.PerKeyCombineFnRunners;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.PaneInfo;
+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.PCollectionView;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * Special version of {@link FlinkReduceFunction} that supports merging windows. This
+ * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the
+ * same behaviour as {@code MergeOverlappingIntervalWindows}.
+ */
+public class FlinkMergingReduceFunction<K, AccumT, OutputT, W extends IntervalWindow>
+    extends FlinkReduceFunction<K, AccumT, OutputT, W> {
+
+  public FlinkMergingReduceFunction(
+      CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> keyedCombineFn,
+      WindowingStrategy<?, W> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions) {
+    super(keyedCombineFn, windowingStrategy, sideInputs, pipelineOptions);
+  }
+
+  @Override
+  public void reduce(
+      Iterable<WindowedValue<KV<K, AccumT>>> elements,
+      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
+
+    PipelineOptions options = serializedOptions.getPipelineOptions();
+
+    FlinkSideInputReader sideInputReader =
+        new FlinkSideInputReader(sideInputs, getRuntimeContext());
+
+    PerKeyCombineFnRunner<K, ?, AccumT, OutputT> combineFnRunner =
+        PerKeyCombineFnRunners.create(combineFn);
+
+    @SuppressWarnings("unchecked")
+    OutputTimeFn<? super BoundedWindow> outputTimeFn =
+        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
+
+    // get all elements so that we can sort them, has to fit into
+    // memory
+    // this seems very unprudent, but correct, for now
+    ArrayList<WindowedValue<KV<K, AccumT>>> sortedInput = Lists.newArrayList();
+    for (WindowedValue<KV<K, AccumT>> inputValue : elements) {
+      for (WindowedValue<KV<K, AccumT>> exploded : inputValue.explodeWindows()) {
+        sortedInput.add(exploded);
+      }
+    }
+    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, AccumT>>>() {
+      @Override
+      public int compare(
+          WindowedValue<KV<K, AccumT>> o1,
+          WindowedValue<KV<K, AccumT>> o2) {
+        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
+            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
+      }
+    });
+
+    // merge windows, we have to do it in an extra pre-processing step and
+    // can't do it as we go since the window of early elements would not
+    // be correct when calling the CombineFn
+    mergeWindow(sortedInput);
+
+    // iterate over the elements that are sorted by window timestamp
+    final Iterator<WindowedValue<KV<K, AccumT>>> iterator = sortedInput.iterator();
+
+    // get the first accumulator
+    WindowedValue<KV<K, AccumT>> currentValue = iterator.next();
+    K key = currentValue.getValue().getKey();
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows());
+    AccumT accumulator = currentValue.getValue().getValue();
+
+    // we use this to keep track of the timestamps assigned by the OutputTimeFn,
+    // in FlinkPartialReduceFunction we already merge the timestamps assigned
+    // to individual elements, here we just merge them
+    List<Instant> windowTimestamps = new ArrayList<>();
+    windowTimestamps.add(currentValue.getTimestamp());
+
+    while (iterator.hasNext()) {
+      WindowedValue<KV<K, AccumT>> nextValue = iterator.next();
+      IntervalWindow nextWindow =
+          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+
+      if (nextWindow.equals(currentWindow)) {
+        // continue accumulating and merge windows
+
+        accumulator = combineFnRunner.mergeAccumulators(
+            key, ImmutableList.of(accumulator, nextValue.getValue().getValue()),
+            options, sideInputReader, currentValue.getWindows());
+
+        windowTimestamps.add(nextValue.getTimestamp());
+      } else {
+        out.collect(
+            WindowedValue.of(
+                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                    options, sideInputReader, currentValue.getWindows())),
+                outputTimeFn.merge(currentWindow, windowTimestamps),
+                currentWindow,
+                PaneInfo.NO_FIRING));
+
+        windowTimestamps.clear();
+
+        currentWindow = nextWindow;
+        currentValue = nextValue;
+        accumulator = nextValue.getValue().getValue();
+        windowTimestamps.add(nextValue.getTimestamp());
+      }
+    }
+
+    // emit the final accumulator
+    out.collect(
+        WindowedValue.of(
+            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                options, sideInputReader, currentValue.getWindows())),
+            outputTimeFn.merge(currentWindow, windowTimestamps),
+            currentWindow,
+            PaneInfo.NO_FIRING));
+  }
+
+  /**
+   * Merge windows. This assumes that the list of elements is sorted by window-end timestamp.
+   * This replaces windows in the input list.
+   */
+  private void mergeWindow(List<WindowedValue<KV<K, AccumT>>> elements) {
+    int currentStart = 0;
+    IntervalWindow currentWindow =
+        (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows());
+
+    for (int i = 1; i < elements.size(); i++) {
+      WindowedValue<KV<K, AccumT>> nextValue = elements.get(i);
+      IntervalWindow nextWindow =
+          (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows());
+      if (currentWindow.intersects(nextWindow)) {
+        // we continue
+        currentWindow = currentWindow.span(nextWindow);
+      } else {
+        // retrofit the merged window to all windows up to "currentStart"
+        for (int j = i - 1; j >= currentStart; j--) {
+          WindowedValue<KV<K, AccumT>> value = elements.get(j);
+          elements.set(
+              j,
+              WindowedValue.of(
+                  value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+        }
+        currentStart = i;
+        currentWindow = nextWindow;
+      }
+    }
+    if (currentStart < elements.size() - 1) {
+      // we have to retrofit the last batch
+      for (int j = elements.size() - 1; j >= currentStart; j--) {
+        WindowedValue<KV<K, AccumT>> value = elements.get(j);
+        elements.set(
+            j,
+            WindowedValue.of(
+                value.getValue(), value.getTimestamp(), currentWindow, value.getPane()));
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
new file mode 100644
index 0000000..9071cc5
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.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 org.apache.beam.runners.flink.translation.functions;
+
+import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * A {@link FlatMapFunction} function that filters out those elements that don't belong in this
+ * output. We need this to implement MultiOutput ParDo functions in combination with
+ * {@link FlinkDoFnFunction}.
+ */
+public class FlinkMultiOutputPruningFunction<T>
+    implements FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<T>> {
+
+  private final int ourOutputTag;
+
+  public FlinkMultiOutputPruningFunction(int ourOutputTag) {
+    this.ourOutputTag = ourOutputTag;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void flatMap(
+      WindowedValue<RawUnionValue> windowedValue,
+      Collector<WindowedValue<T>> collector) throws Exception {
+    int unionTag = windowedValue.getValue().getUnionTag();
+    if (unionTag == ourOutputTag) {
+      collector.collect(
+          (WindowedValue<T>) windowedValue.withValue(windowedValue.getValue().getValue()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java
new file mode 100644
index 0000000..847a00a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.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 org.apache.beam.runners.flink.translation.functions;
+
+import java.io.IOException;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A {@link StepContext} for Flink Batch Runner execution.
+ */
+public class FlinkNoOpStepContext implements StepContext {
+
+  @Override
+  public String getStepName() {
+    return null;
+  }
+
+  @Override
+  public String getTransformName() {
+    return null;
+  }
+
+  @Override
+  public void noteOutput(WindowedValue<?> output) {
+
+  }
+
+  @Override
+  public void noteOutput(TupleTag<?> tag, WindowedValue<?> output) {
+
+  }
+
+  @Override
+  public <T, W extends BoundedWindow> void writePCollectionViewData(
+      TupleTag<?> tag,
+      Iterable<WindowedValue<T>> data,
+      Coder<Iterable<WindowedValue<T>>> dataCoder,
+      W window,
+      Coder<W> windowCoder) throws IOException {
+  }
+
+  @Override
+  public StateInternals<?> stateInternals() {
+    return null;
+  }
+
+  @Override
+  public TimerInternals timerInternals() {
+    return null;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
new file mode 100644
index 0000000..1d1ff9f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import org.apache.beam.runners.core.PerKeyCombineFnRunner;
+import org.apache.beam.runners.core.PerKeyCombineFnRunners;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.flink.api.common.functions.RichGroupCombineFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * This is is the first step for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey}
+ * on Flink. The second part is {@link FlinkReduceFunction}. This function performs a local
+ * combine step before shuffling while the latter does the final combination after a shuffle.
+ *
+ * <p>The input to {@link #combine(Iterable, Collector)} are elements of the same key but
+ * for different windows. We have to ensure that we only combine elements of matching
+ * windows.
+ */
+public class FlinkPartialReduceFunction<K, InputT, AccumT, W extends BoundedWindow>
+    extends RichGroupCombineFunction<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, AccumT>>> {
+
+  protected final CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn;
+
+  protected final WindowingStrategy<?, W> windowingStrategy;
+
+  protected final SerializedPipelineOptions serializedOptions;
+
+  protected final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
+
+  public FlinkPartialReduceFunction(
+      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, ?> combineFn,
+      WindowingStrategy<?, W> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions) {
+
+    this.combineFn = combineFn;
+    this.windowingStrategy = windowingStrategy;
+    this.sideInputs = sideInputs;
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+
+  }
+
+  @Override
+  public void combine(
+      Iterable<WindowedValue<KV<K, InputT>>> elements,
+      Collector<WindowedValue<KV<K, AccumT>>> out) throws Exception {
+
+    PipelineOptions options = serializedOptions.getPipelineOptions();
+
+    FlinkSideInputReader sideInputReader =
+        new FlinkSideInputReader(sideInputs, getRuntimeContext());
+
+    PerKeyCombineFnRunner<K, InputT, AccumT, ?> combineFnRunner =
+        PerKeyCombineFnRunners.create(combineFn);
+
+    @SuppressWarnings("unchecked")
+    OutputTimeFn<? super BoundedWindow> outputTimeFn =
+        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
+
+    // get all elements so that we can sort them, has to fit into
+    // memory
+    // this seems very unprudent, but correct, for now
+    ArrayList<WindowedValue<KV<K, InputT>>> sortedInput = Lists.newArrayList();
+    for (WindowedValue<KV<K, InputT>> inputValue : elements) {
+      for (WindowedValue<KV<K, InputT>> exploded : inputValue.explodeWindows()) {
+        sortedInput.add(exploded);
+      }
+    }
+    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, InputT>>>() {
+      @Override
+      public int compare(
+          WindowedValue<KV<K, InputT>> o1,
+          WindowedValue<KV<K, InputT>> o2) {
+        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
+            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
+      }
+    });
+
+    // iterate over the elements that are sorted by window timestamp
+    //
+    final Iterator<WindowedValue<KV<K, InputT>>> iterator = sortedInput.iterator();
+
+    // create accumulator using the first elements key
+    WindowedValue<KV<K, InputT>> currentValue = iterator.next();
+    K key = currentValue.getValue().getKey();
+    BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null);
+    InputT firstValue = currentValue.getValue().getValue();
+    AccumT accumulator = combineFnRunner.createAccumulator(key,
+        options, sideInputReader, currentValue.getWindows());
+    accumulator = combineFnRunner.addInput(key, accumulator, firstValue,
+        options, sideInputReader, currentValue.getWindows());
+
+    // we use this to keep track of the timestamps assigned by the OutputTimeFn
+    Instant windowTimestamp =
+        outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow);
+
+    while (iterator.hasNext()) {
+      WindowedValue<KV<K, InputT>> nextValue = iterator.next();
+      BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows());
+
+      if (nextWindow.equals(currentWindow)) {
+        // continue accumulating
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+
+        windowTimestamp = outputTimeFn.combine(
+            windowTimestamp,
+            outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow));
+
+      } else {
+        // emit the value that we currently have
+        out.collect(
+            WindowedValue.of(
+                KV.of(key, accumulator),
+                windowTimestamp,
+                currentWindow,
+                PaneInfo.NO_FIRING));
+
+        currentWindow = nextWindow;
+        currentValue = nextValue;
+        InputT value = nextValue.getValue().getValue();
+        accumulator = combineFnRunner.createAccumulator(key,
+            options, sideInputReader, currentValue.getWindows());
+        accumulator = combineFnRunner.addInput(key, accumulator, value,
+            options, sideInputReader, currentValue.getWindows());
+        windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow);
+      }
+    }
+
+    // emit the final accumulator
+    out.collect(
+        WindowedValue.of(
+            KV.of(key, accumulator),
+            windowTimestamp,
+            currentWindow,
+            PaneInfo.NO_FIRING));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
new file mode 100644
index 0000000..3e4f742
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.PerKeyCombineFnRunner;
+import org.apache.beam.runners.core.PerKeyCombineFnRunners;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * This is the second part for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey}
+ * on Flink, the second part is {@link FlinkReduceFunction}. This function performs the final
+ * combination of the pre-combined values after a shuffle.
+ *
+ * <p>The input to {@link #reduce(Iterable, Collector)} are elements of the same key but
+ * for different windows. We have to ensure that we only combine elements of matching
+ * windows.
+ */
+public class FlinkReduceFunction<K, AccumT, OutputT, W extends BoundedWindow>
+    extends RichGroupReduceFunction<WindowedValue<KV<K, AccumT>>, WindowedValue<KV<K, OutputT>>> {
+
+  protected final CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> combineFn;
+
+  protected final WindowingStrategy<?, W> windowingStrategy;
+
+  protected final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
+
+  protected final SerializedPipelineOptions serializedOptions;
+
+  public FlinkReduceFunction(
+      CombineFnBase.PerKeyCombineFn<K, ?, AccumT, OutputT> keyedCombineFn,
+      WindowingStrategy<?, W> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions) {
+
+    this.combineFn = keyedCombineFn;
+
+    this.windowingStrategy = windowingStrategy;
+    this.sideInputs = sideInputs;
+
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+
+  }
+
+  @Override
+  public void reduce(
+      Iterable<WindowedValue<KV<K, AccumT>>> elements,
+      Collector<WindowedValue<KV<K, OutputT>>> out) throws Exception {
+
+    PipelineOptions options = serializedOptions.getPipelineOptions();
+
+    FlinkSideInputReader sideInputReader =
+        new FlinkSideInputReader(sideInputs, getRuntimeContext());
+
+    PerKeyCombineFnRunner<K, ?, AccumT, OutputT> combineFnRunner =
+        PerKeyCombineFnRunners.create(combineFn);
+
+    @SuppressWarnings("unchecked")
+    OutputTimeFn<? super BoundedWindow> outputTimeFn =
+        (OutputTimeFn<? super BoundedWindow>) windowingStrategy.getOutputTimeFn();
+
+
+    // get all elements so that we can sort them, has to fit into
+    // memory
+    // this seems very unprudent, but correct, for now
+    ArrayList<WindowedValue<KV<K, AccumT>>> sortedInput = Lists.newArrayList();
+    for (WindowedValue<KV<K, AccumT>> inputValue: elements) {
+      for (WindowedValue<KV<K, AccumT>> exploded: inputValue.explodeWindows()) {
+        sortedInput.add(exploded);
+      }
+    }
+    Collections.sort(sortedInput, new Comparator<WindowedValue<KV<K, AccumT>>>() {
+      @Override
+      public int compare(
+          WindowedValue<KV<K, AccumT>> o1,
+          WindowedValue<KV<K, AccumT>> o2) {
+        return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp()
+            .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp());
+      }
+    });
+
+    // iterate over the elements that are sorted by window timestamp
+    //
+    final Iterator<WindowedValue<KV<K, AccumT>>> iterator = sortedInput.iterator();
+
+    // get the first accumulator
+    WindowedValue<KV<K, AccumT>> currentValue = iterator.next();
+    K key = currentValue.getValue().getKey();
+    BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null);
+    AccumT accumulator = currentValue.getValue().getValue();
+
+    // we use this to keep track of the timestamps assigned by the OutputTimeFn,
+    // in FlinkPartialReduceFunction we already merge the timestamps assigned
+    // to individual elements, here we just merge them
+    List<Instant> windowTimestamps = new ArrayList<>();
+    windowTimestamps.add(currentValue.getTimestamp());
+
+    while (iterator.hasNext()) {
+      WindowedValue<KV<K, AccumT>> nextValue = iterator.next();
+      BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows());
+
+      if (nextWindow.equals(currentWindow)) {
+        // continue accumulating
+        accumulator = combineFnRunner.mergeAccumulators(
+            key, ImmutableList.of(accumulator, nextValue.getValue().getValue()),
+            options, sideInputReader, currentValue.getWindows());
+
+        windowTimestamps.add(nextValue.getTimestamp());
+      } else {
+        // emit the value that we currently have
+        out.collect(
+            WindowedValue.of(
+                KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                    options, sideInputReader, currentValue.getWindows())),
+                outputTimeFn.merge(currentWindow, windowTimestamps),
+                currentWindow,
+                PaneInfo.NO_FIRING));
+
+        windowTimestamps.clear();
+
+        currentWindow = nextWindow;
+        currentValue = nextValue;
+        accumulator = nextValue.getValue().getValue();
+        windowTimestamps.add(nextValue.getTimestamp());
+      }
+
+    }
+
+    // emit the final accumulator
+    out.collect(
+        WindowedValue.of(
+            KV.of(key, combineFnRunner.extractOutput(key, accumulator,
+                options, sideInputReader, currentValue.getWindows())),
+            outputTimeFn.merge(currentWindow, windowTimestamps),
+            currentWindow,
+            PaneInfo.NO_FIRING));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
new file mode 100644
index 0000000..c317182
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.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 org.apache.beam.runners.flink.translation.functions;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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 org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/**
+ * A {@link SideInputReader} for the Flink Batch Runner.
+ */
+public class FlinkSideInputReader implements SideInputReader {
+
+  private final Map<TupleTag<?>, WindowingStrategy<?, ?>> sideInputs;
+
+  private RuntimeContext runtimeContext;
+
+  public FlinkSideInputReader(Map<PCollectionView<?>, WindowingStrategy<?, ?>> indexByView,
+                              RuntimeContext runtimeContext) {
+    sideInputs = new HashMap<>();
+    for (Map.Entry<PCollectionView<?>, WindowingStrategy<?, ?>> entry : indexByView.entrySet()) {
+      sideInputs.put(entry.getKey().getTagInternal(), entry.getValue());
+    }
+    this.runtimeContext = runtimeContext;
+  }
+
+  @Nullable
+  @Override
+  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+    checkNotNull(view, "View passed to sideInput cannot be null");
+    TupleTag<Iterable<WindowedValue<?>>> tag = view.getTagInternal();
+    checkNotNull(
+        sideInputs.get(tag),
+        "Side input for " + view + " not available.");
+
+    Map<BoundedWindow, T> sideInputs =
+        runtimeContext.getBroadcastVariableWithInitializer(
+            tag.getId(), new SideInputInitializer<>(view));
+    T result = sideInputs.get(window);
+    if (result == null) {
+      result = view.getViewFn().apply(Collections.<WindowedValue<?>>emptyList());
+    }
+    return result;
+  }
+
+  @Override
+  public <T> boolean contains(PCollectionView<T> view) {
+    return sideInputs.containsKey(view.getTagInternal());
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return sideInputs.isEmpty();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
new file mode 100644
index 0000000..c8193d2
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.InMemoryStateInternals;
+import org.apache.beam.runners.core.InMemoryTimerInternals;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaces;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+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.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+/**
+ * A {@link RichGroupReduceFunction} for stateful {@link ParDo} in Flink Batch Runner.
+ */
+public class FlinkStatefulDoFnFunction<K, V, OutputT>
+    extends RichGroupReduceFunction<WindowedValue<KV<K, V>>, WindowedValue<OutputT>> {
+
+  private final DoFn<KV<K, V>, OutputT> dofn;
+  private final WindowingStrategy<?, ?> windowingStrategy;
+  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
+  private final SerializedPipelineOptions serializedOptions;
+  private final Map<TupleTag<?>, Integer> outputMap;
+  private final TupleTag<OutputT> mainOutputTag;
+  private transient DoFnInvoker doFnInvoker;
+
+  public FlinkStatefulDoFnFunction(
+      DoFn<KV<K, V>, OutputT> dofn,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions pipelineOptions,
+      Map<TupleTag<?>, Integer> outputMap,
+      TupleTag<OutputT> mainOutputTag) {
+
+    this.dofn = dofn;
+    this.windowingStrategy = windowingStrategy;
+    this.sideInputs = sideInputs;
+    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
+    this.outputMap = outputMap;
+    this.mainOutputTag = mainOutputTag;
+  }
+
+  @Override
+  public void reduce(
+      Iterable<WindowedValue<KV<K, V>>> values,
+      Collector<WindowedValue<OutputT>> out) throws Exception {
+    RuntimeContext runtimeContext = getRuntimeContext();
+
+    DoFnRunners.OutputManager outputManager;
+    if (outputMap == null) {
+      outputManager = new FlinkDoFnFunction.DoFnOutputManager(out);
+    } else {
+      // it has some additional Outputs
+      outputManager =
+          new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap);
+    }
+
+    final Iterator<WindowedValue<KV<K, V>>> iterator = values.iterator();
+
+    // get the first value, we need this for initializing the state internals with the key.
+    // we are guaranteed to have a first value, otherwise reduce() would not have been called.
+    WindowedValue<KV<K, V>> currentValue = iterator.next();
+    final K key = currentValue.getValue().getKey();
+
+    final InMemoryStateInternals<K> stateInternals = InMemoryStateInternals.forKey(key);
+
+    // Used with Batch, we know that all the data is available for this key. We can't use the
+    // timer manager from the context because it doesn't exist. So we create one and advance
+    // time to the end after processing all elements.
+    final InMemoryTimerInternals timerInternals = new InMemoryTimerInternals();
+    timerInternals.advanceProcessingTime(Instant.now());
+    timerInternals.advanceSynchronizedProcessingTime(Instant.now());
+
+    DoFnRunner<KV<K, V>, OutputT> doFnRunner = DoFnRunners.simpleRunner(
+        serializedOptions.getPipelineOptions(), dofn,
+        new FlinkSideInputReader(sideInputs, runtimeContext),
+        outputManager,
+        mainOutputTag,
+        // see SimpleDoFnRunner, just use it to limit number of additional outputs
+        Collections.<TupleTag<?>>emptyList(),
+        new FlinkNoOpStepContext() {
+          @Override
+          public StateInternals<?> stateInternals() {
+            return stateInternals;
+          }
+          @Override
+          public TimerInternals timerInternals() {
+            return timerInternals;
+          }
+        },
+        new FlinkAggregatorFactory(runtimeContext),
+        windowingStrategy);
+
+    doFnRunner.startBundle();
+
+    doFnRunner.processElement(currentValue);
+    while (iterator.hasNext()) {
+      currentValue = iterator.next();
+      doFnRunner.processElement(currentValue);
+    }
+
+    // Finish any pending windows by advancing the input watermark to infinity.
+    timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    // Finally, advance the processing time to infinity to fire any timers.
+    timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    timerInternals.advanceSynchronizedProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    fireEligibleTimers(timerInternals, doFnRunner);
+
+    doFnRunner.finishBundle();
+  }
+
+  private void fireEligibleTimers(
+      InMemoryTimerInternals timerInternals, DoFnRunner<KV<K, V>, OutputT> runner)
+      throws Exception {
+
+    while (true) {
+
+      TimerInternals.TimerData timer;
+      boolean hasFired = false;
+
+      while ((timer = timerInternals.removeNextEventTimer()) != null) {
+        hasFired = true;
+        fireTimer(timer, runner);
+      }
+      while ((timer = timerInternals.removeNextProcessingTimer()) != null) {
+        hasFired = true;
+        fireTimer(timer, runner);
+      }
+      while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) != null) {
+        hasFired = true;
+        fireTimer(timer, runner);
+      }
+      if (!hasFired) {
+        break;
+      }
+    }
+  }
+
+  private void fireTimer(
+      TimerInternals.TimerData timer, DoFnRunner<KV<K, V>, OutputT> doFnRunner) {
+    StateNamespace namespace = timer.getNamespace();
+    checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+    BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+    doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain());
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    doFnInvoker = DoFnInvokers.invokerFor(dofn);
+    doFnInvoker.invokeSetup();
+  }
+
+  @Override
+  public void close() throws Exception {
+    doFnInvoker.invokeTeardown();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java
new file mode 100644
index 0000000..12222b4
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.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 org.apache.beam.runners.flink.translation.functions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+
+/**
+ * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map}
+ * from window to side input.
+ */
+public class SideInputInitializer<ElemT, ViewT, W extends BoundedWindow>
+    implements BroadcastVariableInitializer<WindowedValue<ElemT>, Map<BoundedWindow, ViewT>> {
+
+  PCollectionView<ViewT> view;
+
+  public SideInputInitializer(PCollectionView<ViewT> view) {
+    this.view = view;
+  }
+
+  @Override
+  public Map<BoundedWindow, ViewT> initializeBroadcastVariable(
+      Iterable<WindowedValue<ElemT>> inputValues) {
+
+    // first partition into windows
+    Map<BoundedWindow, List<WindowedValue<ElemT>>> partitionedElements = new HashMap<>();
+    for (WindowedValue<ElemT> value: inputValues) {
+      for (BoundedWindow window: value.getWindows()) {
+        List<WindowedValue<ElemT>> windowedValues = partitionedElements.get(window);
+        if (windowedValues == null) {
+          windowedValues = new ArrayList<>();
+          partitionedElements.put(window, windowedValues);
+        }
+        windowedValues.add(value);
+      }
+    }
+
+    Map<BoundedWindow, ViewT> resultMap = new HashMap<>();
+
+    for (Map.Entry<BoundedWindow, List<WindowedValue<ElemT>>> elements:
+        partitionedElements.entrySet()) {
+
+      @SuppressWarnings("unchecked")
+      Iterable<WindowedValue<?>> elementsIterable =
+          (List<WindowedValue<?>>) (List<?>) elements.getValue();
+
+      resultMap.put(elements.getKey(), view.getViewFn().apply(elementsIterable));
+    }
+
+    return resultMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
new file mode 100644
index 0000000..9f11212
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.functions;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
new file mode 100644
index 0000000..af4b354
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
new file mode 100644
index 0000000..9b449aa
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.types;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.coders.Coder;
+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;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s.
+ */
+public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
+
+  private final Coder<T> coder;
+
+  public CoderTypeInformation(Coder<T> coder) {
+    checkNotNull(coder);
+    this.coder = coder;
+  }
+
+  public Coder<T> getCoder() {
+    return coder;
+  }
+
+  @Override
+  public boolean isBasicType() {
+    return false;
+  }
+
+  @Override
+  public boolean isTupleType() {
+    return false;
+  }
+
+  @Override
+  public int getArity() {
+    return 1;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Class<T> getTypeClass() {
+    // We don't have the Class, so we have to pass null here. What a shame...
+    return (Class<T>) Object.class;
+  }
+
+  @Override
+  public boolean isKeyType() {
+    return true;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public TypeSerializer<T> createSerializer(ExecutionConfig config) {
+    return new CoderTypeSerializer<>(coder);
+  }
+
+  @Override
+  public int getTotalFields() {
+    return 2;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    CoderTypeInformation that = (CoderTypeInformation) o;
+
+    return coder.equals(that.coder);
+
+  }
+
+  @Override
+  public int hashCode() {
+    return coder.hashCode();
+  }
+
+  @Override
+  public boolean canEqual(Object obj) {
+    return obj instanceof CoderTypeInformation;
+  }
+
+  @Override
+  public String toString() {
+    return "CoderTypeInformation{coder=" + coder + '}';
+  }
+
+  @Override
+  public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
+      executionConfig) {
+    throw new UnsupportedOperationException(
+        "Non-encoded values cannot be compared directly.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
new file mode 100644
index 0000000..e210ed9
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.types;
+
+import java.io.EOFException;
+import java.io.IOException;
+import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}.
+ */
+public class CoderTypeSerializer<T> extends TypeSerializer<T> {
+
+  private Coder<T> coder;
+
+  public CoderTypeSerializer(Coder<T> coder) {
+    this.coder = coder;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @Override
+  public CoderTypeSerializer<T> duplicate() {
+    return new CoderTypeSerializer<>(coder);
+  }
+
+  @Override
+  public T createInstance() {
+    return null;
+  }
+
+  @Override
+  public T copy(T t) {
+    try {
+      return CoderUtils.clone(coder, t);
+    } catch (CoderException e) {
+      throw new RuntimeException("Could not clone.", e);
+    }
+  }
+
+  @Override
+  public T copy(T t, T reuse) {
+    return copy(t);
+  }
+
+  @Override
+  public int getLength() {
+    return -1;
+  }
+
+  @Override
+  public void serialize(T t, DataOutputView dataOutputView) throws IOException {
+    DataOutputViewWrapper outputWrapper = new DataOutputViewWrapper(dataOutputView);
+    coder.encode(t, outputWrapper, Coder.Context.NESTED);
+  }
+
+  @Override
+  public T deserialize(DataInputView dataInputView) throws IOException {
+    try {
+      DataInputViewWrapper inputWrapper = new DataInputViewWrapper(dataInputView);
+      return coder.decode(inputWrapper, Coder.Context.NESTED);
+    } catch (CoderException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof EOFException) {
+        throw (EOFException) cause;
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  @Override
+  public T deserialize(T t, DataInputView dataInputView) throws IOException {
+    return deserialize(dataInputView);
+  }
+
+  @Override
+  public void copy(
+      DataInputView dataInputView,
+      DataOutputView dataOutputView) throws IOException {
+    serialize(deserialize(dataInputView), dataOutputView);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    CoderTypeSerializer that = (CoderTypeSerializer) o;
+    return coder.equals(that.coder);
+  }
+
+  @Override
+  public boolean canEqual(Object obj) {
+    return obj instanceof CoderTypeSerializer;
+  }
+
+  @Override
+  public int hashCode() {
+    return coder.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
new file mode 100644
index 0000000..667ef45
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.types;
+
+import java.io.IOException;
+import java.util.Arrays;
+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;
+import org.apache.flink.core.memory.MemorySegment;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have
+ * been encoded to byte data by a {@link Coder}.
+ */
+public class EncodedValueComparator extends TypeComparator<byte[]> {
+
+  /** For storing the Reference in encoded form. */
+  private transient byte[] encodedReferenceKey;
+
+  private final boolean ascending;
+
+  public EncodedValueComparator(boolean ascending) {
+    this.ascending = ascending;
+  }
+
+  @Override
+  public int hash(byte[] record) {
+    return Arrays.hashCode(record);
+  }
+
+  @Override
+  public void setReference(byte[] toCompare) {
+    this.encodedReferenceKey = toCompare;
+  }
+
+  @Override
+  public boolean equalToReference(byte[] candidate) {
+    if (encodedReferenceKey.length != candidate.length) {
+      return false;
+    }
+    int len = candidate.length;
+    for (int i = 0; i < len; i++) {
+      if (encodedReferenceKey[i] != candidate[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int compareToReference(TypeComparator<byte[]> other) {
+    // VERY IMPORTANT: compareToReference does not behave like Comparable.compare
+    // the meaning of the return value is inverted.
+
+    EncodedValueComparator otherEncodedValueComparator = (EncodedValueComparator) other;
+
+    int len = Math.min(
+        encodedReferenceKey.length,
+        otherEncodedValueComparator.encodedReferenceKey.length);
+
+    for (int i = 0; i < len; i++) {
+      byte b1 = encodedReferenceKey[i];
+      byte b2 = otherEncodedValueComparator.encodedReferenceKey[i];
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? -result : result;
+      }
+    }
+    int result =
+        encodedReferenceKey.length - otherEncodedValueComparator.encodedReferenceKey.length;
+    return ascending ? -result : result;
+  }
+
+
+  @Override
+  public int compare(byte[] first, byte[] second) {
+    int len = Math.min(first.length, second.length);
+    for (int i = 0; i < len; i++) {
+      byte b1 = first[i];
+      byte b2 = second[i];
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? result : -result;
+      }
+    }
+    int result = first.length - second.length;
+    return ascending ? result : -result;
+  }
+
+  @Override
+  public int compareSerialized(
+      DataInputView firstSource,
+      DataInputView secondSource) throws IOException {
+    int lengthFirst = firstSource.readInt();
+    int lengthSecond = secondSource.readInt();
+
+    int len = Math.min(lengthFirst, lengthSecond);
+    for (int i = 0; i < len; i++) {
+      byte b1 = firstSource.readByte();
+      byte b2 = secondSource.readByte();
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? result : -result;
+      }
+    }
+
+    int result = lengthFirst - lengthSecond;
+    return ascending ? result : -result;
+  }
+
+
+
+  @Override
+  public boolean supportsNormalizedKey() {
+    // disabled because this seems to not work with some coders,
+    // such as the AvroCoder
+    return false;
+  }
+
+  @Override
+  public boolean supportsSerializationWithKeyNormalization() {
+    return false;
+  }
+
+  @Override
+  public int getNormalizeKeyLen() {
+    return Integer.MAX_VALUE;
+  }
+
+  @Override
+  public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+    return true;
+  }
+
+  @Override
+  public void putNormalizedKey(byte[] record, MemorySegment target, int offset, int numBytes) {
+    final int limit = offset + numBytes;
+
+    target.put(offset, record, 0, Math.min(numBytes, record.length));
+
+    offset += record.length;
+
+    while (offset < limit) {
+      target.put(offset++, (byte) 0);
+    }
+  }
+
+  @Override
+  public void writeWithKeyNormalization(byte[] record, DataOutputView target) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte[] readWithKeyDenormalization(byte[] reuse, DataInputView source) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean invertNormalizedKey() {
+    return !ascending;
+  }
+
+  @Override
+  public TypeComparator<byte[]> duplicate() {
+    return new EncodedValueComparator(ascending);
+  }
+
+  @Override
+  public int extractKeys(Object record, Object[] target, int index) {
+    target[index] = record;
+    return 1;
+  }
+
+  @Override
+  public TypeComparator[] getFlatComparators() {
+    return new TypeComparator[] { this.duplicate() };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
new file mode 100644
index 0000000..41db61e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.types;
+
+import java.io.IOException;
+
+import org.apache.beam.sdk.coders.Coder;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+/**
+ * {@link TypeSerializer} for values that were encoded using a {@link Coder}.
+ */
+public final class EncodedValueSerializer extends TypeSerializer<byte[]> {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final byte[] EMPTY = new byte[0];
+
+  @Override
+  public boolean isImmutableType() {
+    return true;
+  }
+
+  @Override
+  public byte[] createInstance() {
+    return EMPTY;
+  }
+
+  @Override
+  public byte[] copy(byte[] from) {
+    return from;
+  }
+
+  @Override
+  public byte[] copy(byte[] from, byte[] reuse) {
+    return copy(from);
+  }
+
+  @Override
+  public int getLength() {
+    return -1;
+  }
+
+
+  @Override
+  public void serialize(byte[] record, DataOutputView target) throws IOException {
+    if (record == null) {
+      throw new IllegalArgumentException("The record must not be null.");
+    }
+
+    final int len = record.length;
+    target.writeInt(len);
+    target.write(record);
+  }
+
+  @Override
+  public byte[] deserialize(DataInputView source) throws IOException {
+    final int len = source.readInt();
+    byte[] result = new byte[len];
+    source.readFully(result);
+    return result;
+  }
+
+  @Override
+  public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException {
+    return deserialize(source);
+  }
+
+  @Override
+  public void copy(DataInputView source, DataOutputView target) throws IOException {
+    final int len = source.readInt();
+    target.writeInt(len);
+    target.write(source, len);
+  }
+
+  @Override
+  public boolean canEqual(Object obj) {
+    return obj instanceof EncodedValueSerializer;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.getClass().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj instanceof EncodedValueSerializer;
+  }
+
+  @Override
+  public TypeSerializer<byte[]> duplicate() {
+    return this;
+  }
+}


[09/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java
deleted file mode 100644
index 3a08088..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.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 org.apache.beam.runners.flink.streaming;
-
-import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-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;
-import org.slf4j.LoggerFactory;
-
-/**
- * An unbounded source for testing the unbounded sources framework code.
- *
- * <p>Each split of this sources produces records of the form KV(split_id, i),
- * where i counts up from 0.  Each record has a timestamp of i, and the watermark
- * accurately tracks these timestamps.  The reader will occasionally return false
- * from {@code advance}, in order to simulate a source where not all the data is
- * available immediately.
- */
-public class TestCountingSource
-    extends UnboundedSource<KV<Integer, Integer>, TestCountingSource.CounterMark> {
-  private static final Logger LOG = LoggerFactory.getLogger(TestCountingSource.class);
-
-  private static List<Integer> finalizeTracker;
-  private final int numMessagesPerShard;
-  private final int shardNumber;
-  private final boolean dedup;
-  private final boolean throwOnFirstSnapshot;
-  private final boolean allowSplitting;
-
-  /**
-   * We only allow an exception to be thrown from getCheckpointMark
-   * at most once. This must be static since the entire TestCountingSource
-   * instance may re-serialized when the pipeline recovers and retries.
-   */
-  private static boolean thrown = false;
-
-  public static void setFinalizeTracker(List<Integer> finalizeTracker) {
-    TestCountingSource.finalizeTracker = finalizeTracker;
-  }
-
-  public TestCountingSource(int numMessagesPerShard) {
-    this(numMessagesPerShard, 0, false, false, true);
-  }
-
-  public TestCountingSource withDedup() {
-    return new TestCountingSource(
-        numMessagesPerShard, shardNumber, true, throwOnFirstSnapshot, true);
-  }
-
-  private TestCountingSource withShardNumber(int shardNumber) {
-    return new TestCountingSource(
-        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, true);
-  }
-
-  public TestCountingSource withThrowOnFirstSnapshot(boolean throwOnFirstSnapshot) {
-    return new TestCountingSource(
-        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, true);
-  }
-
-  public TestCountingSource withoutSplitting() {
-    return new TestCountingSource(
-        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, false);
-  }
-
-  private TestCountingSource(int numMessagesPerShard, int shardNumber, boolean dedup,
-                             boolean throwOnFirstSnapshot, boolean allowSplitting) {
-    this.numMessagesPerShard = numMessagesPerShard;
-    this.shardNumber = shardNumber;
-    this.dedup = dedup;
-    this.throwOnFirstSnapshot = throwOnFirstSnapshot;
-    this.allowSplitting = allowSplitting;
-  }
-
-  public int getShardNumber() {
-    return shardNumber;
-  }
-
-  @Override
-  public List<TestCountingSource> split(
-      int desiredNumSplits, PipelineOptions options) {
-    List<TestCountingSource> splits = new ArrayList<>();
-    int numSplits = allowSplitting ? desiredNumSplits : 1;
-    for (int i = 0; i < numSplits; i++) {
-      splits.add(withShardNumber(i));
-    }
-    return splits;
-  }
-
-  class CounterMark implements UnboundedSource.CheckpointMark {
-    int current;
-
-    public CounterMark(int current) {
-      this.current = current;
-    }
-
-    @Override
-    public void finalizeCheckpoint() {
-      if (finalizeTracker != null) {
-        finalizeTracker.add(current);
-      }
-    }
-  }
-
-  @Override
-  public Coder<CounterMark> getCheckpointMarkCoder() {
-    return DelegateCoder.of(
-        VarIntCoder.of(),
-        new DelegateCoder.CodingFunction<CounterMark, Integer>() {
-          @Override
-          public Integer apply(CounterMark input) {
-            return input.current;
-          }
-        },
-        new DelegateCoder.CodingFunction<Integer, CounterMark>() {
-          @Override
-          public CounterMark apply(Integer input) {
-            return new CounterMark(input);
-          }
-        });
-  }
-
-  @Override
-  public boolean requiresDeduping() {
-    return dedup;
-  }
-
-  /**
-   * Public only so that the checkpoint can be conveyed from {@link #getCheckpointMark()} to
-   * {@link TestCountingSource#createReader(PipelineOptions, CounterMark)} without cast.
-   */
-  public class CountingSourceReader extends UnboundedReader<KV<Integer, Integer>> {
-    private int current;
-
-    public CountingSourceReader(int startingPoint) {
-      this.current = startingPoint;
-    }
-
-    @Override
-    public boolean start() {
-      return advance();
-    }
-
-    @Override
-    public boolean advance() {
-      if (current >= numMessagesPerShard - 1) {
-        return false;
-      }
-      // If testing dedup, occasionally insert a duplicate value;
-      if (current >= 0 && dedup && ThreadLocalRandom.current().nextInt(5) == 0) {
-        return true;
-      }
-      current++;
-      return true;
-    }
-
-    @Override
-    public KV<Integer, Integer> getCurrent() {
-      return KV.of(shardNumber, current);
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() {
-      return new Instant(current);
-    }
-
-    @Override
-    public byte[] getCurrentRecordId() {
-      try {
-        return encodeToByteArray(KvCoder.of(VarIntCoder.of(), VarIntCoder.of()), getCurrent());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override
-    public void close() {}
-
-    @Override
-    public TestCountingSource getCurrentSource() {
-      return TestCountingSource.this;
-    }
-
-    @Override
-    public Instant getWatermark() {
-      // The watermark is a promise about future elements, and the timestamps of elements are
-      // strictly increasing for this source.
-      return new Instant(current + 1);
-    }
-
-    @Override
-    public CounterMark getCheckpointMark() {
-      if (throwOnFirstSnapshot && !thrown) {
-        thrown = true;
-        LOG.error("Throwing exception while checkpointing counter");
-        throw new RuntimeException("failed during checkpoint");
-      }
-      // The checkpoint can assume all records read, including the current, have
-      // been commited.
-      return new CounterMark(current);
-    }
-
-    @Override
-    public long getSplitBacklogBytes() {
-      return 7L;
-    }
-  }
-
-  @Override
-  public CountingSourceReader createReader(
-      PipelineOptions options, @Nullable CounterMark checkpointMark) {
-    if (checkpointMark == null) {
-      LOG.debug("creating reader");
-    } else {
-      LOG.debug("restoring reader from checkpoint with current = {}", checkpointMark.current);
-    }
-    return new CountingSourceReader(checkpointMark != null ? checkpointMark.current : -1);
-  }
-
-  @Override
-  public void validate() {}
-
-  @Override
-  public Coder<KV<Integer, Integer>> getDefaultOutputCoder() {
-    return KvCoder.of(VarIntCoder.of(), VarIntCoder.of());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 9e6bba8..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.common.base.Joiner;
-import java.io.Serializable;
-import java.util.Arrays;
-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 org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-
-/**
- * Session window test.
- */
-public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
-  protected String resultPath;
-
-  public TopWikipediaSessionsITCase(){
-  }
-
-  static final String[] EXPECTED_RESULT = new String[] {
-      "user: user1 value:3",
-      "user: user1 value:1",
-      "user: user2 value:4",
-      "user: user2 value:6",
-      "user: user3 value:7",
-      "user: user3 value:2"
-  };
-
-  @Override
-  protected void preSubmit() throws Exception {
-    resultPath = getTempDirPath("result");
-  }
-
-  @Override
-  protected void postSubmit() throws Exception {
-    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-  }
-
-  @Override
-  protected void testProgram() throws Exception {
-
-    Pipeline p = FlinkTestPipeline.createForStreaming();
-
-    Long now = (System.currentTimeMillis() + 10000) / 1000;
-
-    PCollection<KV<String, Long>> output =
-      p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
-          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
-          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
-          ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
-          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
-          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
-          ("contributor_username", "user2"), new TableRow().set("timestamp", now)
-          .set("contributor_username", "user3"))))
-
-
-
-      .apply(ParDo.of(new DoFn<TableRow, String>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) throws Exception {
-          TableRow row = c.element();
-          long timestamp = (Integer) row.get("timestamp");
-          String userName = (String) row.get("contributor_username");
-          if (userName != null) {
-            // Sets the timestamp field to be used in windowing.
-            c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
-          }
-        }
-      }))
-
-      .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
-
-      .apply(Count.<String>perElement());
-
-    PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
-      @ProcessElement
-      public void processElement(ProcessContext c) throws Exception {
-        KV<String, Long> el = c.element();
-        String out = "user: " + el.getKey() + " value:" + el.getValue();
-        c.output(out);
-      }
-    }));
-
-    format.apply(TextIO.Write.to(resultPath));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
deleted file mode 100644
index 90f95d6..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
+++ /dev/null
@@ -1,464 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.state.StateInitializationContext;
-import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Test;
-import org.junit.experimental.runners.Enclosed;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.mockito.Matchers;
-
-/**
- * Tests for {@link UnboundedSourceWrapper}.
- */
-@RunWith(Enclosed.class)
-public class UnboundedSourceWrapperTest {
-
-  /**
-   * Parameterized tests.
-   */
-  @RunWith(Parameterized.class)
-  public static class UnboundedSourceWrapperTestWithParams {
-    private final int numTasks;
-    private final int numSplits;
-
-    public UnboundedSourceWrapperTestWithParams(int numTasks, int numSplits) {
-      this.numTasks = numTasks;
-      this.numSplits = numSplits;
-    }
-
-    @Parameterized.Parameters
-    public static Collection<Object[]> data() {
-      /*
-       * Parameters for initializing the tests:
-       * {numTasks, numSplits}
-       * The test currently assumes powers of two for some assertions.
-       */
-      return Arrays.asList(new Object[][]{
-          {1, 1}, {1, 2}, {1, 4},
-          {2, 1}, {2, 2}, {2, 4},
-          {4, 1}, {4, 2}, {4, 4}
-      });
-    }
-
-    /**
-     * Creates a {@link UnboundedSourceWrapper} that has one or multiple readers per source.
-     * If numSplits > numTasks the source has one source will manage multiple readers.
-     */
-    @Test
-    public void testReaders() throws Exception {
-      final int numElements = 20;
-      final Object checkpointLock = new Object();
-      PipelineOptions options = PipelineOptionsFactory.create();
-
-      // this source will emit exactly NUM_ELEMENTS across all parallel readers,
-      // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
-      // elements later.
-      TestCountingSource source = new TestCountingSource(numElements);
-      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
-          new UnboundedSourceWrapper<>(options, source, numSplits);
-
-      assertEquals(numSplits, flinkWrapper.getSplitSources().size());
-
-      StreamSource<WindowedValue<
-          KV<Integer, Integer>>,
-          UnboundedSourceWrapper<
-              KV<Integer, Integer>,
-              TestCountingSource.CounterMark>> sourceOperator = new StreamSource<>(flinkWrapper);
-
-      setupSourceOperator(sourceOperator, numTasks);
-
-      try {
-        sourceOperator.open();
-        sourceOperator.run(checkpointLock,
-            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
-              private int count = 0;
-
-              @Override
-              public void emitWatermark(Watermark watermark) {
-              }
-
-              @Override
-              public void emitLatencyMarker(LatencyMarker latencyMarker) {
-              }
-
-              @Override
-              public void collect(
-                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
-
-                count++;
-                if (count >= numElements) {
-                  throw new SuccessException();
-                }
-              }
-
-              @Override
-              public void close() {
-
-              }
-            });
-      } catch (SuccessException e) {
-
-        assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
-
-        // success
-        return;
-      }
-      fail("Read terminated without producing expected number of outputs");
-    }
-
-    /**
-     * Verify that snapshot/restore work as expected. We bring up a source and cancel
-     * after seeing a certain number of elements. Then we snapshot that source,
-     * bring up a completely new source that we restore from the snapshot and verify
-     * that we see all expected elements in the end.
-     */
-    @Test
-    public void testRestore() throws Exception {
-      final int numElements = 20;
-      final Object checkpointLock = new Object();
-      PipelineOptions options = PipelineOptionsFactory.create();
-
-      // this source will emit exactly NUM_ELEMENTS across all parallel readers,
-      // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
-      // elements later.
-      TestCountingSource source = new TestCountingSource(numElements);
-      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
-          new UnboundedSourceWrapper<>(options, source, numSplits);
-
-      assertEquals(numSplits, flinkWrapper.getSplitSources().size());
-
-      StreamSource<
-          WindowedValue<KV<Integer, Integer>>,
-          UnboundedSourceWrapper<
-              KV<Integer, Integer>,
-              TestCountingSource.CounterMark>> sourceOperator = new StreamSource<>(flinkWrapper);
-
-
-      OperatorStateStore backend = mock(OperatorStateStore.class);
-
-      TestingListState<KV<UnboundedSource, TestCountingSource.CounterMark>>
-          listState = new TestingListState<>();
-
-      when(backend.getOperatorState(Matchers.any(ListStateDescriptor.class)))
-          .thenReturn(listState);
-
-      StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-      when(initializationContext.getOperatorStateStore()).thenReturn(backend);
-      when(initializationContext.isRestored()).thenReturn(false, true);
-
-      flinkWrapper.initializeState(initializationContext);
-
-      setupSourceOperator(sourceOperator, numTasks);
-
-      final Set<KV<Integer, Integer>> emittedElements = new HashSet<>();
-
-      boolean readFirstBatchOfElements = false;
-
-      try {
-        sourceOperator.open();
-        sourceOperator.run(checkpointLock,
-            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
-              private int count = 0;
-
-              @Override
-              public void emitWatermark(Watermark watermark) {
-              }
-
-              @Override
-              public void emitLatencyMarker(LatencyMarker latencyMarker) {
-              }
-
-              @Override
-              public void collect(
-                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
-
-                emittedElements.add(windowedValueStreamRecord.getValue().getValue());
-                count++;
-                if (count >= numElements / 2) {
-                  throw new SuccessException();
-                }
-              }
-
-              @Override
-              public void close() {
-
-              }
-            });
-      } catch (SuccessException e) {
-        // success
-        readFirstBatchOfElements = true;
-      }
-
-      assertTrue("Did not successfully read first batch of elements.", readFirstBatchOfElements);
-
-      // draw a snapshot
-      flinkWrapper.snapshotState(new StateSnapshotContextSynchronousImpl(0, 0));
-
-      // test snapshot offsets
-      assertEquals(flinkWrapper.getLocalSplitSources().size(),
-          listState.getList().size());
-      int totalEmit = 0;
-      for (KV<UnboundedSource, TestCountingSource.CounterMark> kv : listState.get()) {
-        totalEmit += kv.getValue().current + 1;
-      }
-      assertEquals(numElements / 2, totalEmit);
-
-      // test that finalizeCheckpoint on CheckpointMark is called
-      final ArrayList<Integer> finalizeList = new ArrayList<>();
-      TestCountingSource.setFinalizeTracker(finalizeList);
-      flinkWrapper.notifyCheckpointComplete(0);
-      assertEquals(flinkWrapper.getLocalSplitSources().size(), finalizeList.size());
-
-      // create a completely new source but restore from the snapshot
-      TestCountingSource restoredSource = new TestCountingSource(numElements);
-      UnboundedSourceWrapper<
-          KV<Integer, Integer>, TestCountingSource.CounterMark> restoredFlinkWrapper =
-          new UnboundedSourceWrapper<>(options, restoredSource, numSplits);
-
-      assertEquals(numSplits, restoredFlinkWrapper.getSplitSources().size());
-
-      StreamSource<
-          WindowedValue<KV<Integer, Integer>>,
-          UnboundedSourceWrapper<
-              KV<Integer, Integer>,
-              TestCountingSource.CounterMark>> restoredSourceOperator =
-          new StreamSource<>(restoredFlinkWrapper);
-
-      setupSourceOperator(restoredSourceOperator, numTasks);
-
-      // restore snapshot
-      restoredFlinkWrapper.initializeState(initializationContext);
-
-      boolean readSecondBatchOfElements = false;
-
-      // run again and verify that we see the other elements
-      try {
-        restoredSourceOperator.open();
-        restoredSourceOperator.run(checkpointLock,
-            new Output<StreamRecord<WindowedValue<KV<Integer, Integer>>>>() {
-              private int count = 0;
-
-              @Override
-              public void emitWatermark(Watermark watermark) {
-              }
-
-              @Override
-              public void emitLatencyMarker(LatencyMarker latencyMarker) {
-              }
-
-              @Override
-              public void collect(
-                  StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
-                emittedElements.add(windowedValueStreamRecord.getValue().getValue());
-                count++;
-                if (count >= numElements / 2) {
-                  throw new SuccessException();
-                }
-              }
-
-              @Override
-              public void close() {
-
-              }
-            });
-      } catch (SuccessException e) {
-        // success
-        readSecondBatchOfElements = true;
-      }
-
-      assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
-
-      assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements);
-
-      // verify that we saw all NUM_ELEMENTS elements
-      assertTrue(emittedElements.size() == numElements);
-    }
-
-    @Test
-    public void testNullCheckpoint() throws Exception {
-      final int numElements = 20;
-      PipelineOptions options = PipelineOptionsFactory.create();
-
-      TestCountingSource source = new TestCountingSource(numElements) {
-        @Override
-        public Coder<CounterMark> getCheckpointMarkCoder() {
-          return null;
-        }
-      };
-      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
-          new UnboundedSourceWrapper<>(options, source, numSplits);
-
-      OperatorStateStore backend = mock(OperatorStateStore.class);
-
-      TestingListState<KV<UnboundedSource, TestCountingSource.CounterMark>>
-          listState = new TestingListState<>();
-
-      when(backend.getOperatorState(Matchers.any(ListStateDescriptor.class)))
-          .thenReturn(listState);
-
-      StateInitializationContext initializationContext = mock(StateInitializationContext.class);
-
-      when(initializationContext.getOperatorStateStore()).thenReturn(backend);
-      when(initializationContext.isRestored()).thenReturn(false, true);
-
-      flinkWrapper.initializeState(initializationContext);
-
-      StreamSource sourceOperator = new StreamSource<>(flinkWrapper);
-      setupSourceOperator(sourceOperator, numTasks);
-      sourceOperator.open();
-
-      flinkWrapper.snapshotState(new StateSnapshotContextSynchronousImpl(0, 0));
-
-      assertEquals(0, listState.getList().size());
-
-      UnboundedSourceWrapper<
-          KV<Integer, Integer>, TestCountingSource.CounterMark> restoredFlinkWrapper =
-          new UnboundedSourceWrapper<>(options, new TestCountingSource(numElements),
-              numSplits);
-
-      StreamSource restoredSourceOperator = new StreamSource<>(flinkWrapper);
-      setupSourceOperator(restoredSourceOperator, numTasks);
-      sourceOperator.open();
-
-      restoredFlinkWrapper.initializeState(initializationContext);
-
-      assertEquals(Math.max(1, numSplits / numTasks), flinkWrapper.getLocalSplitSources().size());
-
-    }
-
-    @SuppressWarnings("unchecked")
-    private static <T> void setupSourceOperator(StreamSource<T, ?> operator, int numSubTasks) {
-      ExecutionConfig executionConfig = new ExecutionConfig();
-      StreamConfig cfg = new StreamConfig(new Configuration());
-
-      cfg.setTimeCharacteristic(TimeCharacteristic.EventTime);
-
-      Environment env = new DummyEnvironment("MockTwoInputTask", numSubTasks, 0);
-
-      StreamTask<?, ?> mockTask = mock(StreamTask.class);
-      when(mockTask.getName()).thenReturn("Mock Task");
-      when(mockTask.getCheckpointLock()).thenReturn(new Object());
-      when(mockTask.getConfiguration()).thenReturn(cfg);
-      when(mockTask.getEnvironment()).thenReturn(env);
-      when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
-      when(mockTask.getAccumulatorMap())
-          .thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap());
-      TestProcessingTimeService testProcessingTimeService = new TestProcessingTimeService();
-      when(mockTask.getProcessingTimeService()).thenReturn(testProcessingTimeService);
-
-      operator.setup(mockTask, cfg, (Output<StreamRecord<T>>) mock(Output.class));
-    }
-
-    /**
-     * A special {@link RuntimeException} that we throw to signal that the test was successful.
-     */
-    private static class SuccessException extends RuntimeException {
-    }
-  }
-
-  /**
-   * Not parameterized tests.
-   */
-  public static class BasicTest {
-
-    /**
-     * Check serialization a {@link UnboundedSourceWrapper}.
-     */
-    @Test
-    public void testSerialization() throws Exception {
-      final int parallelism = 1;
-      final int numElements = 20;
-      PipelineOptions options = PipelineOptionsFactory.create();
-
-      TestCountingSource source = new TestCountingSource(numElements);
-      UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
-          new UnboundedSourceWrapper<>(options, source, parallelism);
-
-      InstantiationUtil.serializeObject(flinkWrapper);
-    }
-
-  }
-
-  private static final class TestingListState<T> implements ListState<T> {
-
-    private final List<T> list = new ArrayList<>();
-
-    @Override
-    public void clear() {
-      list.clear();
-    }
-
-    @Override
-    public Iterable<T> get() throws Exception {
-      return list;
-    }
-
-    @Override
-    public void add(T value) throws Exception {
-      list.add(value);
-    }
-
-    public List<T> getList() {
-      return list;
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
deleted file mode 100644
index 08a1e03..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.streaming;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/resources/log4j-test.properties b/runners/flink/runner/src/test/resources/log4j-test.properties
deleted file mode 100644
index 4c74d85..0000000
--- a/runners/flink/runner/src/test/resources/log4j-test.properties
+++ /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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-log4j.rootLogger=OFF, testlogger
-
-# A1 is set to be a ConsoleAppender.
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java
new file mode 100644
index 0000000..b745f0b
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.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 org.apache.beam.runners.flink;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.GlobalConfiguration;
+
+/**
+ * {@link DefaultValueFactory} for getting a default value for the parallelism option
+ * on {@link FlinkPipelineOptions}.
+ *
+ * <p>This will return either the default value from {@link GlobalConfiguration} or {@code 1}.
+ * A valid {@link GlobalConfiguration} is only available if the program is executed by the Flink
+ * run scripts.
+ */
+public class DefaultParallelismFactory implements DefaultValueFactory<Integer> {
+  @Override
+  public Integer create(PipelineOptions options) {
+    return GlobalConfiguration.loadConfiguration()
+        .getInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
new file mode 100644
index 0000000..854b674
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a
+ * Flink batch job.
+ */
+class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FlinkBatchPipelineTranslator.class);
+
+  /**
+   * The necessary context in the case of a batch job.
+   */
+  private final FlinkBatchTranslationContext batchContext;
+
+  private int depth = 0;
+
+  public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
+    this.batchContext = new FlinkBatchTranslationContext(env, options);
+  }
+
+  @Override
+  @SuppressWarnings("rawtypes, unchecked")
+  public void translate(Pipeline pipeline) {
+    super.translate(pipeline);
+
+    // terminate dangling DataSets
+    for (DataSet<?> dataSet: batchContext.getDanglingDataSets().values()) {
+      dataSet.output(new DiscardingOutputFormat());
+    }
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Pipeline Visitor Methods
+  // --------------------------------------------------------------------------------------------
+
+  @Override
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
+    LOG.info("{} enterCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
+    this.depth++;
+
+    BatchTransformTranslator<?> translator = getTranslator(node);
+
+    if (translator != null) {
+      applyBatchTransform(node.getTransform(), node, translator);
+      LOG.info("{} translated- {}", genSpaces(this.depth), node.getFullName());
+      return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+    } else {
+      return CompositeBehavior.ENTER_TRANSFORM;
+    }
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
+    this.depth--;
+    LOG.info("{} leaveCompositeTransform- {}", genSpaces(this.depth), node.getFullName());
+  }
+
+  @Override
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+    LOG.info("{} visitPrimitiveTransform- {}", genSpaces(this.depth), node.getFullName());
+
+    // get the transformation corresponding to the node we are
+    // currently visiting and translate it into its Flink alternative.
+    PTransform<?, ?> transform = node.getTransform();
+    BatchTransformTranslator<?> translator =
+        FlinkBatchTransformTranslators.getTranslator(transform);
+    if (translator == null) {
+      LOG.info(node.getTransform().getClass().toString());
+      throw new UnsupportedOperationException("The transform " + transform
+          + " is currently not supported.");
+    }
+    applyBatchTransform(transform, node, translator);
+  }
+
+  private <T extends PTransform<?, ?>> void applyBatchTransform(
+      PTransform<?, ?> transform,
+      TransformHierarchy.Node node,
+      BatchTransformTranslator<?> translator) {
+
+    @SuppressWarnings("unchecked")
+    T typedTransform = (T) transform;
+
+    @SuppressWarnings("unchecked")
+    BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
+
+    // create the applied PTransform on the batchContext
+    batchContext.setCurrentTransform(node.toAppliedPTransform());
+    typedTranslator.translateNode(typedTransform, batchContext);
+  }
+
+  /**
+   * A translator of a {@link PTransform}.
+   */
+  public interface BatchTransformTranslator<TransformT extends PTransform> {
+    void translateNode(TransformT transform, FlinkBatchTranslationContext context);
+  }
+
+  /**
+   * Returns a translator for the given node, if it is possible, otherwise null.
+   */
+  private static BatchTransformTranslator<?> getTranslator(TransformHierarchy.Node node) {
+    PTransform<?, ?> transform = node.getTransform();
+
+    // Root of the graph is null
+    if (transform == null) {
+      return null;
+    }
+
+    return FlinkBatchTransformTranslators.getTranslator(transform);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
new file mode 100644
index 0000000..ff9521c
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
@@ -0,0 +1,723 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
+import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMergingPartialReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMergingReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkStatefulDoFnFunction;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.KvKeySelector;
+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.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+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.transforms.Combine;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+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.join.RawUnionValue;
+import org.apache.beam.sdk.transforms.join.UnionCoder;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+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.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.Reshuffle;
+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 org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+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.api.java.DataSet;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.GroupCombineOperator;
+import org.apache.flink.api.java.operators.GroupReduceOperator;
+import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.MapPartitionOperator;
+import org.apache.flink.api.java.operators.SingleInputUdfOperator;
+import org.apache.flink.util.Collector;
+
+/**
+ * Translators for transforming {@link PTransform PTransforms} to
+ * Flink {@link DataSet DataSets}.
+ */
+class FlinkBatchTransformTranslators {
+
+  // --------------------------------------------------------------------------------------------
+  //  Transform Translator Registry
+  // --------------------------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private static final Map<
+      Class<? extends PTransform>,
+      FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
+
+  static {
+    TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
+
+    TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
+    TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
+    TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorBatch());
+
+    TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslatorBatch());
+
+    TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch());
+
+    TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoTranslatorBatch());
+
+    TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
+  }
+
+
+  static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(
+      PTransform<?, ?> transform) {
+    return TRANSLATORS.get(transform.getClass());
+  }
+
+  private static class ReadSourceTranslatorBatch<T>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
+
+    @Override
+    public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
+      String name = transform.getName();
+      BoundedSource<T> source = transform.getSource();
+      PCollection<T> output = context.getOutput(transform);
+
+      TypeInformation<WindowedValue<T>> typeInformation = context.getTypeInfo(output);
+
+      DataSource<WindowedValue<T>> dataSource = new DataSource<>(
+          context.getExecutionEnvironment(),
+          new SourceInputFormat<>(source, context.getPipelineOptions()),
+          typeInformation,
+          name);
+
+      context.setOutputDataSet(output, dataSource);
+    }
+  }
+
+  private static class WindowAssignTranslatorBatch<T>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Window.Assign<T>> {
+
+    @Override
+    public void translateNode(Window.Assign<T> transform, FlinkBatchTranslationContext context) {
+      PValue input = context.getInput(transform);
+
+      TypeInformation<WindowedValue<T>> resultTypeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+      DataSet<WindowedValue<T>> inputDataSet = context.getInputDataSet(input);
+
+      @SuppressWarnings("unchecked")
+      final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
+          (WindowingStrategy<T, ? extends BoundedWindow>)
+              context.getOutput(transform).getWindowingStrategy();
+
+      WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
+
+      FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
+          new FlinkAssignWindows<>(windowFn);
+
+      DataSet<WindowedValue<T>> resultDataSet = inputDataSet
+          .flatMap(assignWindowsFunction)
+          .name(context.getOutput(transform).getName())
+          .returns(resultTypeInfo);
+
+      context.setOutputDataSet(context.getOutput(transform), resultDataSet);
+    }
+  }
+
+  private static class GroupByKeyTranslatorBatch<K, InputT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, InputT>> {
+
+    @Override
+    public void translateNode(
+        GroupByKey<K, InputT> transform,
+        FlinkBatchTranslationContext context) {
+
+      // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API
+      // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn
+
+      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+
+      Combine.KeyedCombineFn<K, InputT, List<InputT>, List<InputT>> combineFn =
+          new Concatenate<InputT>().asKeyedFn();
+
+      KvCoder<K, InputT> inputCoder =
+          (KvCoder<K, InputT>) context.getInput(transform).getCoder();
+
+      Coder<List<InputT>> accumulatorCoder;
+
+      try {
+        accumulatorCoder =
+            combineFn.getAccumulatorCoder(
+                context.getInput(transform).getPipeline().getCoderRegistry(),
+                inputCoder.getKeyCoder(),
+                inputCoder.getValueCoder());
+      } catch (CannotProvideCoderException e) {
+        throw new RuntimeException(e);
+      }
+
+      WindowingStrategy<?, ?> windowingStrategy =
+          context.getInput(transform).getWindowingStrategy();
+
+      TypeInformation<WindowedValue<KV<K, List<InputT>>>> partialReduceTypeInfo =
+          new CoderTypeInformation<>(
+              WindowedValue.getFullCoder(
+                  KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
+                  windowingStrategy.getWindowFn().windowCoder()));
+
+
+      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
+          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
+
+      FlinkPartialReduceFunction<K, InputT, List<InputT>, ?> partialReduceFunction;
+      FlinkReduceFunction<K, List<InputT>, List<InputT>, ?> reduceFunction;
+
+      if (windowingStrategy.getWindowFn().isNonMerging()) {
+        @SuppressWarnings("unchecked")
+        WindowingStrategy<?, BoundedWindow> boundedStrategy =
+            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;
+
+        partialReduceFunction = new FlinkPartialReduceFunction<>(
+            combineFn,
+            boundedStrategy,
+            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
+            context.getPipelineOptions());
+
+        reduceFunction = new FlinkReduceFunction<>(
+            combineFn,
+            boundedStrategy,
+            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
+            context.getPipelineOptions());
+
+      } else {
+        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
+          throw new UnsupportedOperationException(
+              "Merging WindowFn with windows other than IntervalWindow are not supported.");
+        }
+
+        @SuppressWarnings("unchecked")
+        WindowingStrategy<?, IntervalWindow> intervalStrategy =
+            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;
+
+        partialReduceFunction = new FlinkMergingPartialReduceFunction<>(
+            combineFn,
+            intervalStrategy,
+            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
+            context.getPipelineOptions());
+
+        reduceFunction = new FlinkMergingReduceFunction<>(
+            combineFn,
+            intervalStrategy,
+            Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(),
+            context.getPipelineOptions());
+      }
+
+      // Partially GroupReduce the values into the intermediate format AccumT (combine)
+      GroupCombineOperator<
+          WindowedValue<KV<K, InputT>>,
+          WindowedValue<KV<K, List<InputT>>>> groupCombine =
+          new GroupCombineOperator<>(
+              inputGrouping,
+              partialReduceTypeInfo,
+              partialReduceFunction,
+              "GroupCombine: " + transform.getName());
+
+      Grouping<WindowedValue<KV<K, List<InputT>>>> intermediateGrouping =
+          groupCombine.groupBy(new KvKeySelector<List<InputT>, K>(inputCoder.getKeyCoder()));
+
+      // Fully reduce the values and create output format VO
+      GroupReduceOperator<
+          WindowedValue<KV<K, List<InputT>>>, WindowedValue<KV<K, List<InputT>>>> outputDataSet =
+          new GroupReduceOperator<>(
+              intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName());
+
+      context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+
+    }
+
+  }
+
+  private static class ReshuffleTranslatorBatch<K, InputT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Reshuffle<K, InputT>> {
+
+    @Override
+    public void translateNode(
+        Reshuffle<K, InputT> transform,
+        FlinkBatchTranslationContext context) {
+
+      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+
+      context.setOutputDataSet(context.getOutput(transform), inputDataSet.rebalance());
+
+    }
+
+  }
+
+  /**
+   * Combiner that combines {@code T}s into a single {@code List<T>} containing all inputs.
+   *
+   * <p>For internal use to translate {@link GroupByKey}. For a large {@link PCollection} this
+   * is expected to crash!
+   *
+   * <p>This is copied from the dataflow runner code.
+   *
+   * @param <T> the type of elements to concatenate.
+   */
+  private static class Concatenate<T> extends Combine.CombineFn<T, List<T>, List<T>> {
+    @Override
+    public List<T> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<T> addInput(List<T> accumulator, T input) {
+      accumulator.add(input);
+      return accumulator;
+    }
+
+    @Override
+    public List<T> mergeAccumulators(Iterable<List<T>> accumulators) {
+      List<T> result = createAccumulator();
+      for (List<T> accumulator : accumulators) {
+        result.addAll(accumulator);
+      }
+      return result;
+    }
+
+    @Override
+    public List<T> extractOutput(List<T> accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<List<T>> getAccumulatorCoder(CoderRegistry registry, Coder<T> inputCoder) {
+      return ListCoder.of(inputCoder);
+    }
+
+    @Override
+    public Coder<List<T>> getDefaultOutputCoder(CoderRegistry registry, Coder<T> inputCoder) {
+      return ListCoder.of(inputCoder);
+    }
+  }
+
+
+  private static class CombinePerKeyTranslatorBatch<K, InputT, AccumT, OutputT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
+          Combine.PerKey<K, InputT, OutputT>> {
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void translateNode(
+        Combine.PerKey<K, InputT, OutputT> transform,
+        FlinkBatchTranslationContext context) {
+      DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+
+      CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT> combineFn =
+          (CombineFnBase.PerKeyCombineFn<K, InputT, AccumT, OutputT>) transform.getFn();
+
+      KvCoder<K, InputT> inputCoder =
+          (KvCoder<K, InputT>) context.getInput(transform).getCoder();
+
+      Coder<AccumT> accumulatorCoder;
+
+      try {
+        accumulatorCoder =
+            combineFn.getAccumulatorCoder(
+                context.getInput(transform).getPipeline().getCoderRegistry(),
+                inputCoder.getKeyCoder(),
+                inputCoder.getValueCoder());
+      } catch (CannotProvideCoderException e) {
+        throw new RuntimeException(e);
+      }
+
+      WindowingStrategy<?, ?> windowingStrategy =
+          context.getInput(transform).getWindowingStrategy();
+
+      TypeInformation<WindowedValue<KV<K, AccumT>>> partialReduceTypeInfo =
+          context.getTypeInfo(
+              KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
+              windowingStrategy);
+
+      Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
+          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
+
+      // construct a map from side input to WindowingStrategy so that
+      // the DoFn runner can map main-input windows to side input windows
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
+      for (PCollectionView<?> sideInput: transform.getSideInputs()) {
+        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
+      }
+
+      if (windowingStrategy.getWindowFn().isNonMerging()) {
+        WindowingStrategy<?, BoundedWindow> boundedStrategy =
+            (WindowingStrategy<?, BoundedWindow>) windowingStrategy;
+
+        FlinkPartialReduceFunction<K, InputT, AccumT, ?> partialReduceFunction =
+            new FlinkPartialReduceFunction<>(
+                combineFn,
+                boundedStrategy,
+                sideInputStrategies,
+                context.getPipelineOptions());
+
+        FlinkReduceFunction<K, AccumT, OutputT, ?> reduceFunction =
+            new FlinkReduceFunction<>(
+                combineFn,
+                boundedStrategy,
+                sideInputStrategies,
+                context.getPipelineOptions());
+
+        // Partially GroupReduce the values into the intermediate format AccumT (combine)
+        GroupCombineOperator<
+            WindowedValue<KV<K, InputT>>,
+            WindowedValue<KV<K, AccumT>>> groupCombine =
+            new GroupCombineOperator<>(
+                inputGrouping,
+                partialReduceTypeInfo,
+                partialReduceFunction,
+                "GroupCombine: " + transform.getName());
+
+        transformSideInputs(transform.getSideInputs(), groupCombine, context);
+
+        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
+            context.getTypeInfo(context.getOutput(transform));
+
+        Grouping<WindowedValue<KV<K, AccumT>>> intermediateGrouping =
+            groupCombine.groupBy(new KvKeySelector<AccumT, K>(inputCoder.getKeyCoder()));
+
+        // Fully reduce the values and create output format OutputT
+        GroupReduceOperator<
+            WindowedValue<KV<K, AccumT>>, WindowedValue<KV<K, OutputT>>> outputDataSet =
+            new GroupReduceOperator<>(
+                intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
+
+        transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+        context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+
+      } else {
+        if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) {
+          throw new UnsupportedOperationException(
+              "Merging WindowFn with windows other than IntervalWindow are not supported.");
+        }
+
+        // for merging windows we can't to a pre-shuffle combine step since
+        // elements would not be in their correct windows for side-input access
+
+        WindowingStrategy<?, IntervalWindow> intervalStrategy =
+            (WindowingStrategy<?, IntervalWindow>) windowingStrategy;
+
+        FlinkMergingNonShuffleReduceFunction<K, InputT, AccumT, OutputT, ?> reduceFunction =
+            new FlinkMergingNonShuffleReduceFunction<>(
+                combineFn,
+                intervalStrategy,
+                sideInputStrategies,
+                context.getPipelineOptions());
+
+        TypeInformation<WindowedValue<KV<K, OutputT>>> reduceTypeInfo =
+            context.getTypeInfo(context.getOutput(transform));
+
+        Grouping<WindowedValue<KV<K, InputT>>> grouping =
+            inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
+
+        // Fully reduce the values and create output format OutputT
+        GroupReduceOperator<
+            WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, OutputT>>> outputDataSet =
+            new GroupReduceOperator<>(
+                grouping, reduceTypeInfo, reduceFunction, transform.getName());
+
+        transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+        context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+      }
+
+
+    }
+  }
+
+  private static void rejectSplittable(DoFn<?, ?> doFn) {
+    DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
+    if (signature.processElement().isSplittable()) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "%s does not currently support splittable DoFn: %s",
+              FlinkRunner.class.getSimpleName(), doFn));
+    }
+  }
+
+  private static class ParDoTranslatorBatch<InputT, OutputT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
+      ParDo.MultiOutput<InputT, OutputT>> {
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void translateNode(
+        ParDo.MultiOutput<InputT, OutputT> transform,
+        FlinkBatchTranslationContext context) {
+      DoFn<InputT, OutputT> doFn = transform.getFn();
+      rejectSplittable(doFn);
+      DataSet<WindowedValue<InputT>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+
+      Map<TupleTag<?>, PValue> outputs = context.getOutputs(transform);
+
+      Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
+      // put the main output at index 0, FlinkMultiOutputDoFnFunction  expects this
+      outputMap.put(transform.getMainOutputTag(), 0);
+      int count = 1;
+      for (TupleTag<?> tag : outputs.keySet()) {
+        if (!outputMap.containsKey(tag)) {
+          outputMap.put(tag, count++);
+        }
+      }
+
+      // assume that the windowing strategy is the same for all outputs
+      WindowingStrategy<?, ?> windowingStrategy = null;
+
+      // collect all output Coders and create a UnionCoder for our tagged outputs
+      List<Coder<?>> outputCoders = Lists.newArrayList();
+      for (PValue taggedValue : outputs.values()) {
+        checkState(
+            taggedValue instanceof PCollection,
+            "Within ParDo, got a non-PCollection output %s of type %s",
+            taggedValue,
+            taggedValue.getClass().getSimpleName());
+        PCollection<?> coll = (PCollection<?>) taggedValue;
+        outputCoders.add(coll.getCoder());
+        windowingStrategy = coll.getWindowingStrategy();
+      }
+
+      if (windowingStrategy == null) {
+        throw new IllegalStateException("No outputs defined.");
+      }
+
+      UnionCoder unionCoder = UnionCoder.of(outputCoders);
+
+      TypeInformation<WindowedValue<RawUnionValue>> typeInformation =
+          new CoderTypeInformation<>(
+              WindowedValue.getFullCoder(
+                  unionCoder,
+                  windowingStrategy.getWindowFn().windowCoder()));
+
+      List<PCollectionView<?>> sideInputs = transform.getSideInputs();
+
+      // construct a map from side input to WindowingStrategy so that
+      // the DoFn runner can map main-input windows to side input windows
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
+      for (PCollectionView<?> sideInput: sideInputs) {
+        sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal());
+      }
+
+      SingleInputUdfOperator<WindowedValue<InputT>, WindowedValue<RawUnionValue>, ?> outputDataSet;
+      DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass());
+      if (signature.stateDeclarations().size() > 0
+          || signature.timerDeclarations().size() > 0) {
+
+        // Based on the fact that the signature is stateful, DoFnSignatures ensures
+        // that it is also keyed
+        KvCoder<?, InputT> inputCoder =
+            (KvCoder<?, InputT>) context.getInput(transform).getCoder();
+
+        FlinkStatefulDoFnFunction<?, ?, OutputT> doFnWrapper = new FlinkStatefulDoFnFunction<>(
+            (DoFn) doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(),
+            outputMap, transform.getMainOutputTag()
+        );
+
+        Grouping<WindowedValue<InputT>> grouping =
+            inputDataSet.groupBy(new KvKeySelector(inputCoder.getKeyCoder()));
+
+        outputDataSet =
+            new GroupReduceOperator(grouping, typeInformation, doFnWrapper, transform.getName());
+
+      } else {
+        FlinkDoFnFunction<InputT, RawUnionValue> doFnWrapper =
+            new FlinkDoFnFunction(
+                doFn,
+                windowingStrategy,
+                sideInputStrategies,
+                context.getPipelineOptions(),
+                outputMap,
+                transform.getMainOutputTag());
+
+        outputDataSet = new MapPartitionOperator<>(
+            inputDataSet, typeInformation,
+            doFnWrapper, transform.getName());
+
+      }
+
+      transformSideInputs(sideInputs, outputDataSet, context);
+
+      for (Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+        pruneOutput(
+            outputDataSet,
+            context,
+            outputMap.get(output.getKey()),
+            (PCollection) output.getValue());
+      }
+
+    }
+
+    private <T> void pruneOutput(
+        DataSet<WindowedValue<RawUnionValue>> taggedDataSet,
+        FlinkBatchTranslationContext context,
+        int integerTag,
+        PCollection<T> collection) {
+      TypeInformation<WindowedValue<T>> outputType = context.getTypeInfo(collection);
+
+      FlinkMultiOutputPruningFunction<T> pruningFunction =
+          new FlinkMultiOutputPruningFunction<>(integerTag);
+
+      FlatMapOperator<WindowedValue<RawUnionValue>, WindowedValue<T>> pruningOperator =
+          new FlatMapOperator<>(
+              taggedDataSet,
+              outputType,
+              pruningFunction,
+              collection.getName());
+
+      context.setOutputDataSet(collection, pruningOperator);
+    }
+  }
+
+  private static class FlattenPCollectionTranslatorBatch<T>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
+      Flatten.PCollections<T>> {
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void translateNode(
+        Flatten.PCollections<T> transform,
+        FlinkBatchTranslationContext context) {
+
+      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
+      DataSet<WindowedValue<T>> result = null;
+
+      if (allInputs.isEmpty()) {
+
+        // create an empty dummy source to satisfy downstream operations
+        // we cannot create an empty source in Flink, therefore we have to
+        // add the flatMap that simply never forwards the single element
+        DataSource<String> dummySource =
+            context.getExecutionEnvironment().fromElements("dummy");
+        result = dummySource.flatMap(new FlatMapFunction<String, WindowedValue<T>>() {
+          @Override
+          public void flatMap(String s, Collector<WindowedValue<T>> collector) throws Exception {
+            // never return anything
+          }
+        }).returns(
+            new CoderTypeInformation<>(
+                WindowedValue.getFullCoder(
+                    (Coder<T>) VoidCoder.of(),
+                    GlobalWindow.Coder.INSTANCE)));
+      } else {
+        for (PValue taggedPc : allInputs.values()) {
+          checkArgument(
+              taggedPc instanceof PCollection,
+              "Got non-PCollection input to flatten: %s of type %s",
+              taggedPc,
+              taggedPc.getClass().getSimpleName());
+          PCollection<T> collection = (PCollection<T>) taggedPc;
+          DataSet<WindowedValue<T>> current = context.getInputDataSet(collection);
+          if (result == null) {
+            result = current;
+          } else {
+            result = result.union(current);
+          }
+        }
+      }
+
+      // insert a dummy filter, there seems to be a bug in Flink
+      // that produces duplicate elements after the union in some cases
+      // if we don't
+      result = result.filter(new FilterFunction<WindowedValue<T>>() {
+        @Override
+        public boolean filter(WindowedValue<T> tWindowedValue) throws Exception {
+          return true;
+        }
+      }).name("UnionFixFilter");
+      context.setOutputDataSet(context.getOutput(transform), result);
+    }
+  }
+
+  private static class CreatePCollectionViewTranslatorBatch<ElemT, ViewT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
+          View.CreatePCollectionView<ElemT, ViewT>> {
+
+    @Override
+    public void translateNode(
+        View.CreatePCollectionView<ElemT, ViewT> transform,
+        FlinkBatchTranslationContext context) {
+      DataSet<WindowedValue<ElemT>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+
+      PCollectionView<ViewT> input = transform.getView();
+
+      context.setSideInputDataSet(input, inputDataSet);
+    }
+  }
+
+  private static void transformSideInputs(
+      List<PCollectionView<?>> sideInputs,
+      SingleInputUdfOperator<?, ?, ?> outputDataSet,
+      FlinkBatchTranslationContext context) {
+    // get corresponding Flink broadcast DataSets
+    for (PCollectionView<?> input : sideInputs) {
+      DataSet<?> broadcastSet = context.getSideInputDataSet(input);
+      outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
+    }
+  }
+
+  private FlinkBatchTransformTranslators() {}
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
new file mode 100644
index 0000000..98dd0fb
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.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 org.apache.beam.runners.flink;
+
+import com.google.common.collect.Iterables;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.coders.Coder;
+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.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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 org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * Helper for {@link FlinkBatchPipelineTranslator} and translators in
+ * {@link FlinkBatchTransformTranslators}.
+ */
+class FlinkBatchTranslationContext {
+
+  private final Map<PValue, DataSet<?>> dataSets;
+  private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
+
+  /**
+   * For keeping track about which DataSets don't have a successor. We
+   * need to terminate these with a discarding sink because the Beam
+   * model allows dangling operations.
+   */
+  private final Map<PValue, DataSet<?>> danglingDataSets;
+
+  private final ExecutionEnvironment env;
+  private final PipelineOptions options;
+
+  private AppliedPTransform<?, ?, ?> currentTransform;
+
+  // ------------------------------------------------------------------------
+
+  public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
+    this.env = env;
+    this.options = options;
+    this.dataSets = new HashMap<>();
+    this.broadcastDataSets = new HashMap<>();
+
+    this.danglingDataSets = new HashMap<>();
+  }
+
+  // ------------------------------------------------------------------------
+
+  public Map<PValue, DataSet<?>> getDanglingDataSets() {
+    return danglingDataSets;
+  }
+
+  public ExecutionEnvironment getExecutionEnvironment() {
+    return env;
+  }
+
+  public PipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> DataSet<WindowedValue<T>> getInputDataSet(PValue value) {
+    // assume that the DataSet is used as an input if retrieved here
+    danglingDataSets.remove(value);
+    return (DataSet<WindowedValue<T>>) dataSets.get(value);
+  }
+
+  public <T> void setOutputDataSet(PValue value, DataSet<WindowedValue<T>> set) {
+    if (!dataSets.containsKey(value)) {
+      dataSets.put(value, set);
+      danglingDataSets.put(value, set);
+    }
+  }
+
+  /**
+   * Sets the AppliedPTransform which carries input/output.
+   * @param currentTransform
+   */
+  public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+    this.currentTransform = currentTransform;
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
+    return (DataSet<T>) broadcastDataSets.get(value);
+  }
+
+  public <ViewT, ElemT> void setSideInputDataSet(
+      PCollectionView<ViewT> value,
+      DataSet<WindowedValue<ElemT>> set) {
+    if (!broadcastDataSets.containsKey(value)) {
+      broadcastDataSets.put(value, set);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
+    return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy());
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(
+      Coder<T> coder,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
+        WindowedValue.getFullCoder(
+            coder,
+            windowingStrategy.getWindowFn().windowCoder());
+
+    return new CoderTypeInformation<>(windowedValueCoder);
+  }
+
+  Map<TupleTag<?>, PValue> getInputs(PTransform<?, ?> transform) {
+    return currentTransform.getInputs();
+  }
+
+  @SuppressWarnings("unchecked")
+  <T extends PValue> T getInput(PTransform<T, ?> transform) {
+    return (T) Iterables.getOnlyElement(currentTransform.getInputs().values());
+  }
+
+  Map<TupleTag<?>, PValue> getOutputs(PTransform<?, ?> transform) {
+    return currentTransform.getOutputs();
+  }
+
+  @SuppressWarnings("unchecked")
+  <T extends PValue> T getOutput(PTransform<?, T> transform) {
+    return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java
new file mode 100644
index 0000000..bf4395f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.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 org.apache.beam.runners.flink;
+
+import java.io.IOException;
+
+import org.apache.beam.sdk.AggregatorRetrievalException;
+import org.apache.beam.sdk.AggregatorValues;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.MetricResults;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.joda.time.Duration;
+
+
+/**
+ * Result of a detached execution of a {@link org.apache.beam.sdk.Pipeline} with Flink.
+ * In detached execution, results and job execution are currently unavailable.
+ */
+public class FlinkDetachedRunnerResult implements PipelineResult {
+
+  FlinkDetachedRunnerResult() {}
+
+  @Override
+  public State getState() {
+    return State.UNKNOWN;
+  }
+
+  @Override
+  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
+      throws AggregatorRetrievalException {
+    throw new AggregatorRetrievalException(
+        "Accumulators can't be retrieved for detached Job executions.",
+        new UnsupportedOperationException());
+  }
+
+  @Override
+  public MetricResults metrics() {
+    throw new UnsupportedOperationException("The FlinkRunner does not currently support metrics.");
+  }
+
+  @Override
+  public State cancel() throws IOException {
+    throw new UnsupportedOperationException("Cancelling is not yet supported.");
+  }
+
+  @Override
+  public State waitUntilFinish() {
+    return State.UNKNOWN;
+  }
+
+  @Override
+  public State waitUntilFinish(Duration duration) {
+    return State.UNKNOWN;
+  }
+
+  @Override
+  public String toString() {
+    return "FlinkDetachedRunnerResult{}";
+  }
+}


[10/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 44c9017..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.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 org.apache.beam.runners.flink;
-
-import com.google.common.base.Joiner;
-import java.io.File;
-import java.net.URI;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-/**
- * Reads from a bounded source in batch execution.
- */
-public class ReadSourceITCase extends JavaProgramTestBase {
-
-  protected String resultPath;
-
-  public ReadSourceITCase(){
-  }
-
-  private static final String[] EXPECTED_RESULT = new String[] {
-     "0", "1", "2", "3", "4", "5", "6", "7", "8", "9"};
-
-  @Override
-  protected void preSubmit() throws Exception {
-    resultPath = getTempDirPath("result");
-
-    // need to create the dir, otherwise Beam sinks don't
-    // work for these tests
-
-    if (!new File(new URI(resultPath)).mkdirs()) {
-      throw new RuntimeException("Could not create output dir.");
-    }
-  }
-
-  @Override
-  protected void postSubmit() throws Exception {
-    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-  }
-
-  @Override
-  protected void testProgram() throws Exception {
-    runProgram(resultPath);
-  }
-
-  private static void runProgram(String resultPath) throws Exception {
-
-    Pipeline p = FlinkTestPipeline.createForBatch();
-
-    PCollection<String> result = p
-        .apply(CountingInput.upTo(10))
-        .apply(ParDo.of(new DoFn<Long, String>() {
-          @ProcessElement
-          public void processElement(ProcessContext c) throws Exception {
-            c.output(c.element().toString());
-          }
-        }));
-
-    result.apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part"));
-
-    p.run();
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java
deleted file mode 100644
index 79b7882..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.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 org.apache.beam.runners.flink;
-
-import com.google.common.base.Joiner;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-/**
- * Reads from a bounded source in streaming.
- */
-public class ReadSourceStreamingITCase extends StreamingProgramTestBase {
-
-  protected String resultPath;
-
-  public ReadSourceStreamingITCase(){
-  }
-
-  private static final String[] EXPECTED_RESULT = new String[] {
-     "0", "1", "2", "3", "4", "5", "6", "7", "8", "9"};
-
-  @Override
-  protected void preSubmit() throws Exception {
-    resultPath = getTempDirPath("result");
-  }
-
-  @Override
-  protected void postSubmit() throws Exception {
-    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-  }
-
-  @Override
-  protected void testProgram() throws Exception {
-    runProgram(resultPath);
-  }
-
-  private static void runProgram(String resultPath) {
-
-    Pipeline p = FlinkTestPipeline.createForStreaming();
-
-    p
-      .apply(CountingInput.upTo(10))
-      .apply(ParDo.of(new DoFn<Long, String>() {
-          @ProcessElement
-          public void processElement(ProcessContext c) throws Exception {
-            c.output(c.element().toString());
-          }
-        }))
-      .apply(TextIO.Write.to(resultPath));
-
-    p.run();
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 38b790e..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.flink;
-
-import static org.junit.Assert.assertNotNull;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.URI;
-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.io.Write;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-/**
- * Tests the translation of custom Write sinks.
- */
-public class WriteSinkITCase extends JavaProgramTestBase {
-
-  protected String resultPath;
-
-  public WriteSinkITCase(){
-  }
-
-  static final String[] EXPECTED_RESULT = new String[] {
-      "Joe red 3", "Mary blue 4", "Max yellow 23"};
-
-  @Override
-  protected void preSubmit() throws Exception {
-    resultPath = getTempDirPath("result-" + System.nanoTime());
-  }
-
-  @Override
-  protected void postSubmit() throws Exception {
-    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-  }
-
-  @Override
-  protected void testProgram() throws Exception {
-    runProgram(resultPath);
-  }
-
-  @Override
-  public void stopCluster() throws Exception {
-    try {
-      super.stopCluster();
-    } catch (final IOException ioe) {
-      if (ioe.getMessage().startsWith("Unable to delete file")) {
-        // that's ok for the test itself, just the OS playing with us on cleanup phase
-      }
-    }
-  }
-
-  private static void runProgram(String resultPath) {
-    Pipeline p = FlinkTestPipeline.createForBatch();
-
-    p.apply(Create.of(ImmutableList.copyOf(EXPECTED_RESULT))).setCoder(StringUtf8Coder.of())
-      .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
-
-    p.run();
-  }
-
-  /**
-   * Simple custom sink which writes to a file.
-   */
-  private static class MyCustomSink extends Sink<String> {
-
-    private final String resultPath;
-
-    public MyCustomSink(String resultPath) {
-      this.resultPath = resultPath;
-    }
-
-    @Override
-    public void validate(PipelineOptions options) {
-      assertNotNull(options);
-    }
-
-    @Override
-    public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
-      return new MyWriteOperation();
-    }
-
-    private class MyWriteOperation extends WriteOperation<String, String> {
-
-      @Override
-      public Coder<String> getWriterResultCoder() {
-        return StringUtf8Coder.of();
-      }
-
-      @Override
-      public void initialize(PipelineOptions options) throws Exception {
-
-      }
-
-      @Override
-      public void setWindowedWrites(boolean windowedWrites) {
-
-      }
-
-      @Override
-      public void finalize(Iterable<String> writerResults, PipelineOptions options)
-          throws Exception {
-
-      }
-
-      @Override
-      public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
-        return new MyWriter();
-      }
-
-      @Override
-      public Sink<String> getSink() {
-        return MyCustomSink.this;
-      }
-
-      /**
-       * Simple Writer which writes to a file.
-       */
-      private class MyWriter extends Writer<String, String> {
-
-        private PrintWriter internalWriter;
-
-        @Override
-        public final void openWindowed(String uId,
-                                       BoundedWindow window,
-                                       PaneInfo paneInfo,
-                                       int shard,
-                                       int numShards) throws Exception {
-          throw new UnsupportedOperationException("Windowed writes not supported.");
-        }
-
-        @Override
-        public final void openUnwindowed(String uId, int shard, int numShards) throws Exception {
-          Path path = new Path(resultPath + "/" + uId);
-          FileSystem.get(new URI("file:///")).create(path, false);
-          internalWriter = new PrintWriter(new File(path.toUri()));
-        }
-
-        @Override
-        public void cleanup() throws Exception {
-
-        }
-
-        @Override
-        public void write(String value) throws Exception {
-          internalWriter.println(value);
-        }
-
-        @Override
-        public String close() throws Exception {
-          internalWriter.close();
-          return resultPath;
-        }
-
-        @Override
-        public WriteOperation<String, String> getWriteOperation() {
-          return MyWriteOperation.this;
-        }
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
deleted file mode 100644
index 4c826d1..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
+++ /dev/null
@@ -1,600 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import java.util.Collections;
-import java.util.HashMap;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.StatefulDoFnRunner;
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
-import org.apache.beam.sdk.coders.Coder;
-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.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PCollectionViewTesting;
-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.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.Timer;
-import org.apache.beam.sdk.util.TimerSpec;
-import org.apache.beam.sdk.util.TimerSpecs;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateSpec;
-import org.apache.beam.sdk.util.state.StateSpecs;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link DoFnOperator}.
- */
-@RunWith(JUnit4.class)
-public class DoFnOperatorTest {
-
-  // views and windows for testing side inputs
-  private static final long WINDOW_MSECS_1 = 100;
-  private static final long WINDOW_MSECS_2 = 500;
-
-  private WindowingStrategy<Object, IntervalWindow> windowingStrategy1 =
-      WindowingStrategy.of(FixedWindows.of(new Duration(WINDOW_MSECS_1)));
-
-  private PCollectionView<Iterable<String>> view1 =
-      PCollectionViewTesting.testingView(
-          new TupleTag<Iterable<WindowedValue<String>>>() {},
-          new PCollectionViewTesting.IdentityViewFn<String>(),
-          StringUtf8Coder.of(),
-          windowingStrategy1);
-
-  private WindowingStrategy<Object, IntervalWindow> windowingStrategy2 =
-      WindowingStrategy.of(FixedWindows.of(new Duration(WINDOW_MSECS_2)));
-
-  private PCollectionView<Iterable<String>> view2 =
-      PCollectionViewTesting.testingView(
-          new TupleTag<Iterable<WindowedValue<String>>>() {},
-          new PCollectionViewTesting.IdentityViewFn<String>(),
-          StringUtf8Coder.of(),
-          windowingStrategy2);
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testSingleOutput() throws Exception {
-
-    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
-        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
-
-    TupleTag<String> outputTag = new TupleTag<>("main-output");
-
-    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
-        new IdentityDoFn<String>(),
-        windowedValueCoder,
-        outputTag,
-        Collections.<TupleTag<?>>emptyList(),
-        new DoFnOperator.DefaultOutputManagerFactory(),
-        WindowingStrategy.globalDefault(),
-        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
-        null);
-
-    OneInputStreamOperatorTestHarness<WindowedValue<String>, String> testHarness =
-        new OneInputStreamOperatorTestHarness<>(doFnOperator);
-
-    testHarness.open();
-
-    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("Hello")));
-
-    assertThat(
-        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(WindowedValue.valueInGlobalWindow("Hello")));
-
-    testHarness.close();
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testMultiOutputOutput() throws Exception {
-
-    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
-        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
-
-    TupleTag<String> mainOutput = new TupleTag<>("main-output");
-    TupleTag<String> additionalOutput1 = new TupleTag<>("output-1");
-    TupleTag<String> additionalOutput2 = new TupleTag<>("output-2");
-    ImmutableMap<TupleTag<?>, Integer> outputMapping = ImmutableMap.<TupleTag<?>, Integer>builder()
-        .put(mainOutput, 1)
-        .put(additionalOutput1, 2)
-        .put(additionalOutput2, 3)
-        .build();
-
-    DoFnOperator<String, String, RawUnionValue> doFnOperator = new DoFnOperator<>(
-        new MultiOutputDoFn(additionalOutput1, additionalOutput2),
-        windowedValueCoder,
-        mainOutput,
-        ImmutableList.<TupleTag<?>>of(additionalOutput1, additionalOutput2),
-        new DoFnOperator.MultiOutputOutputManagerFactory(outputMapping),
-        WindowingStrategy.globalDefault(),
-        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
-        null);
-
-    OneInputStreamOperatorTestHarness<WindowedValue<String>, RawUnionValue> testHarness =
-        new OneInputStreamOperatorTestHarness<>(doFnOperator);
-
-    testHarness.open();
-
-    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("one")));
-    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("two")));
-    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("hello")));
-
-    assertThat(
-        this.stripStreamRecordFromRawUnion(testHarness.getOutput()),
-        contains(
-            new RawUnionValue(2, WindowedValue.valueInGlobalWindow("extra: one")),
-            new RawUnionValue(3, WindowedValue.valueInGlobalWindow("extra: two")),
-            new RawUnionValue(1, WindowedValue.valueInGlobalWindow("got: hello")),
-            new RawUnionValue(2, WindowedValue.valueInGlobalWindow("got: hello")),
-            new RawUnionValue(3, WindowedValue.valueInGlobalWindow("got: hello"))));
-
-    testHarness.close();
-  }
-
-  @Test
-  public void testLateDroppingForStatefulFn() throws Exception {
-
-    WindowingStrategy<Object, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(new Duration(10)));
-
-    DoFn<Integer, String> fn = new DoFn<Integer, String>() {
-
-      @StateId("state")
-      private final StateSpec<Object, ValueState<String>> stateSpec =
-          StateSpecs.value(StringUtf8Coder.of());
-
-      @ProcessElement
-      public void processElement(ProcessContext context) {
-        context.output(context.element().toString());
-      }
-    };
-
-    WindowedValue.FullWindowedValueCoder<Integer> windowedValueCoder =
-        WindowedValue.getFullCoder(
-            VarIntCoder.of(),
-            windowingStrategy.getWindowFn().windowCoder());
-
-    TupleTag<String> outputTag = new TupleTag<>("main-output");
-
-    DoFnOperator<Integer, String, WindowedValue<String>> doFnOperator = new DoFnOperator<>(
-        fn,
-        windowedValueCoder,
-        outputTag,
-        Collections.<TupleTag<?>>emptyList(),
-        new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<String>>(),
-        windowingStrategy,
-        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
-        VarIntCoder.of() /* key coder */);
-
-    OneInputStreamOperatorTestHarness<WindowedValue<Integer>, WindowedValue<String>> testHarness =
-        new KeyedOneInputStreamOperatorTestHarness<>(
-            doFnOperator,
-            new KeySelector<WindowedValue<Integer>, Integer>() {
-              @Override
-              public Integer getKey(WindowedValue<Integer> integerWindowedValue) throws Exception {
-                return integerWindowedValue.getValue();
-              }
-            },
-            new CoderTypeInformation<>(VarIntCoder.of()));
-
-    testHarness.open();
-
-    testHarness.processWatermark(0);
-
-    IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10));
-
-    // this should not be late
-    testHarness.processElement(
-        new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING)));
-
-    assertThat(
-        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(WindowedValue.of("13", new Instant(0), window1, PaneInfo.NO_FIRING)));
-
-    testHarness.getOutput().clear();
-
-    testHarness.processWatermark(9);
-
-    // this should still not be considered late
-    testHarness.processElement(
-        new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING)));
-
-    assertThat(
-        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(WindowedValue.of("17", new Instant(0), window1, PaneInfo.NO_FIRING)));
-
-    testHarness.getOutput().clear();
-
-    testHarness.processWatermark(10);
-
-    // this should now be considered late
-    testHarness.processElement(
-        new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING)));
-
-    assertThat(
-        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        emptyIterable());
-
-    testHarness.close();
-  }
-
-  @Test
-  public void testStateGCForStatefulFn() throws Exception {
-
-    WindowingStrategy<Object, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(new Duration(10))).withAllowedLateness(Duration.ZERO);
-
-    final String timerId = "boo";
-    final String stateId = "dazzle";
-
-    final int offset = 5000;
-    final int timerOutput = 4093;
-
-    DoFn<KV<String, Integer>, KV<String, Integer>> fn =
-        new DoFn<KV<String, Integer>, KV<String, Integer>>() {
-
-          @TimerId(timerId)
-          private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
-
-          @StateId(stateId)
-          private final StateSpec<Object, ValueState<String>> stateSpec =
-              StateSpecs.value(StringUtf8Coder.of());
-
-          @ProcessElement
-          public void processElement(
-              ProcessContext context,
-              @TimerId(timerId) Timer timer,
-              @StateId(stateId) ValueState<String> state,
-              BoundedWindow window) {
-            timer.set(window.maxTimestamp());
-            state.write(context.element().getKey());
-            context.output(
-                KV.of(context.element().getKey(), context.element().getValue() + offset));
-          }
-
-          @OnTimer(timerId)
-          public void onTimer(OnTimerContext context, @StateId(stateId) ValueState<String> state) {
-            context.output(KV.of(state.read(), timerOutput));
-          }
-        };
-
-    WindowedValue.FullWindowedValueCoder<KV<String, Integer>> windowedValueCoder =
-        WindowedValue.getFullCoder(
-            KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
-            windowingStrategy.getWindowFn().windowCoder());
-
-    TupleTag<KV<String, Integer>> outputTag = new TupleTag<>("main-output");
-
-    DoFnOperator<
-        KV<String, Integer>, KV<String, Integer>, WindowedValue<KV<String, Integer>>> doFnOperator =
-        new DoFnOperator<>(
-            fn,
-            windowedValueCoder,
-            outputTag,
-            Collections.<TupleTag<?>>emptyList(),
-            new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<String, Integer>>>(),
-            windowingStrategy,
-            new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
-            Collections.<PCollectionView<?>>emptyList(), /* side inputs */
-            PipelineOptionsFactory.as(FlinkPipelineOptions.class),
-            StringUtf8Coder.of() /* key coder */);
-
-    KeyedOneInputStreamOperatorTestHarness<
-        String,
-        WindowedValue<KV<String, Integer>>,
-        WindowedValue<KV<String, Integer>>> testHarness =
-        new KeyedOneInputStreamOperatorTestHarness<>(
-            doFnOperator,
-            new KeySelector<WindowedValue<KV<String, Integer>>, String>() {
-              @Override
-              public String getKey(
-                  WindowedValue<KV<String, Integer>> kvWindowedValue) throws Exception {
-                return kvWindowedValue.getValue().getKey();
-              }
-            },
-            new CoderTypeInformation<>(StringUtf8Coder.of()));
-
-    testHarness.open();
-
-    testHarness.processWatermark(0);
-
-    assertEquals(0, testHarness.numKeyedStateEntries());
-
-    IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10));
-
-    testHarness.processElement(
-        new StreamRecord<>(
-            WindowedValue.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING)));
-
-    testHarness.processElement(
-        new StreamRecord<>(
-            WindowedValue.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING)));
-
-    assertThat(
-        this.<KV<String, Integer>>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(
-            WindowedValue.of(
-                KV.of("key1", 5 + offset), new Instant(1), window1, PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                KV.of("key2", 7 + offset), new Instant(3), window1, PaneInfo.NO_FIRING)));
-
-    assertEquals(2, testHarness.numKeyedStateEntries());
-
-    testHarness.getOutput().clear();
-
-    // this should trigger both the window.maxTimestamp() timer and the GC timer
-    // this tests that the GC timer fires after the user timer
-    testHarness.processWatermark(
-        window1.maxTimestamp()
-            .plus(windowingStrategy.getAllowedLateness())
-            .plus(StatefulDoFnRunner.TimeInternalsCleanupTimer.GC_DELAY_MS)
-            .getMillis());
-
-    assertThat(
-        this.<KV<String, Integer>>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(
-            WindowedValue.of(
-                KV.of("key1", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                KV.of("key2", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING)));
-
-    // ensure the state was garbage collected
-    assertEquals(0, testHarness.numKeyedStateEntries());
-
-    testHarness.close();
-  }
-
-  public void testSideInputs(boolean keyed) throws Exception {
-
-    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
-        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
-
-    TupleTag<String> outputTag = new TupleTag<>("main-output");
-
-    ImmutableMap<Integer, PCollectionView<?>> sideInputMapping =
-        ImmutableMap.<Integer, PCollectionView<?>>builder()
-            .put(1, view1)
-            .put(2, view2)
-            .build();
-
-    Coder<String> keyCoder = null;
-    if (keyed) {
-      keyCoder = StringUtf8Coder.of();
-    }
-
-    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
-        new IdentityDoFn<String>(),
-        windowedValueCoder,
-        outputTag,
-        Collections.<TupleTag<?>>emptyList(),
-        new DoFnOperator.DefaultOutputManagerFactory<String>(),
-        WindowingStrategy.globalDefault(),
-        sideInputMapping, /* side-input mapping */
-        ImmutableList.<PCollectionView<?>>of(view1, view2), /* side inputs */
-        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
-        keyCoder);
-
-    TwoInputStreamOperatorTestHarness<WindowedValue<String>, RawUnionValue, String> testHarness =
-        new TwoInputStreamOperatorTestHarness<>(doFnOperator);
-
-    if (keyed) {
-      // we use a dummy key for the second input since it is considered to be broadcast
-      testHarness = new KeyedTwoInputStreamOperatorTestHarness<>(
-          doFnOperator,
-          new StringKeySelector(),
-          new DummyKeySelector(),
-          BasicTypeInfo.STRING_TYPE_INFO);
-    }
-
-    testHarness.open();
-
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(500));
-
-    // test the keep of sideInputs events
-    testHarness.processElement2(
-        new StreamRecord<>(
-            new RawUnionValue(
-                1,
-                valuesInWindow(ImmutableList.of("hello", "ciao"), new Instant(0), firstWindow))));
-    testHarness.processElement2(
-        new StreamRecord<>(
-            new RawUnionValue(
-                2,
-                valuesInWindow(ImmutableList.of("foo", "bar"), new Instant(0), secondWindow))));
-
-    // push in a regular elements
-    WindowedValue<String> helloElement = valueInWindow("Hello", new Instant(0), firstWindow);
-    WindowedValue<String> worldElement = valueInWindow("World", new Instant(1000), firstWindow);
-    testHarness.processElement1(new StreamRecord<>(helloElement));
-    testHarness.processElement1(new StreamRecord<>(worldElement));
-
-    // test the keep of pushed-back events
-    testHarness.processElement2(
-        new StreamRecord<>(
-            new RawUnionValue(
-                1,
-                valuesInWindow(ImmutableList.of("hello", "ciao"),
-                    new Instant(1000), firstWindow))));
-    testHarness.processElement2(
-        new StreamRecord<>(
-            new RawUnionValue(
-                2,
-                valuesInWindow(ImmutableList.of("foo", "bar"), new Instant(1000), secondWindow))));
-
-    assertThat(
-        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
-        contains(helloElement, worldElement));
-
-    testHarness.close();
-
-  }
-
-  /**
-   * {@link TwoInputStreamOperatorTestHarness} support OperatorStateBackend,
-   * but don't support KeyedStateBackend. So we just test sideInput of normal ParDo.
-   */
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testNormalParDoSideInputs() throws Exception {
-    testSideInputs(false);
-  }
-
-  @Test
-  public void testKeyedSideInputs() throws Exception {
-    testSideInputs(true);
-  }
-
-  private <T> Iterable<WindowedValue<T>> stripStreamRecordFromWindowedValue(
-      Iterable<Object> input) {
-
-    return FluentIterable.from(input).filter(new Predicate<Object>() {
-      @Override
-      public boolean apply(@Nullable Object o) {
-        return o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof WindowedValue;
-      }
-    }).transform(new Function<Object, WindowedValue<T>>() {
-      @Nullable
-      @Override
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      public WindowedValue<T> apply(@Nullable Object o) {
-        if (o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof WindowedValue) {
-          return (WindowedValue) ((StreamRecord) o).getValue();
-        }
-        throw new RuntimeException("unreachable");
-      }
-    });
-  }
-
-  private Iterable<RawUnionValue> stripStreamRecordFromRawUnion(Iterable<Object> input) {
-    return FluentIterable.from(input).filter(new Predicate<Object>() {
-      @Override
-      public boolean apply(@Nullable Object o) {
-        return o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof RawUnionValue;
-      }
-    }).transform(new Function<Object, RawUnionValue>() {
-      @Nullable
-      @Override
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      public RawUnionValue apply(@Nullable Object o) {
-        if (o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof RawUnionValue) {
-          return (RawUnionValue) ((StreamRecord) o).getValue();
-        }
-        throw new RuntimeException("unreachable");
-      }
-    });
-  }
-
-  private static class MultiOutputDoFn extends DoFn<String, String> {
-    private TupleTag<String> additionalOutput1;
-    private TupleTag<String> additionalOutput2;
-
-    public MultiOutputDoFn(TupleTag<String> additionalOutput1, TupleTag<String> additionalOutput2) {
-      this.additionalOutput1 = additionalOutput1;
-      this.additionalOutput2 = additionalOutput2;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      if (c.element().equals("one")) {
-        c.output(additionalOutput1, "extra: one");
-      } else if (c.element().equals("two")) {
-        c.output(additionalOutput2, "extra: two");
-      } else {
-        c.output("got: " + c.element());
-        c.output(additionalOutput1, "got: " + c.element());
-        c.output(additionalOutput2, "got: " + c.element());
-      }
-    }
-  }
-
-  private static class IdentityDoFn<T> extends DoFn<T, T> {
-    @ProcessElement
-    public void processElement(ProcessContext c) throws Exception {
-      c.output(c.element());
-    }
-  }
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private WindowedValue<Iterable<?>> valuesInWindow(
-      Iterable<?> values, Instant timestamp, BoundedWindow window) {
-    return (WindowedValue) WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING);
-  }
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private <T> WindowedValue<T> valueInWindow(
-      T value, Instant timestamp, BoundedWindow window) {
-    return WindowedValue.of(value, timestamp, window, PaneInfo.NO_FIRING);
-  }
-
-
-  private static class DummyKeySelector implements KeySelector<RawUnionValue, String> {
-    @Override
-    public String getKey(RawUnionValue stringWindowedValue) throws Exception {
-      return "dummy_key";
-    }
-  }
-
-  private static class StringKeySelector implements KeySelector<WindowedValue<String>, String> {
-    @Override
-    public String getKey(WindowedValue<String> stringWindowedValue) throws Exception {
-      return stringWindowedValue.getValue();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
deleted file mode 100644
index 7e7d1e1..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertThat;
-
-import java.util.Arrays;
-import org.apache.beam.runners.core.StateMerging;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
-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.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.GroupingState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.state.OperatorStateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlinkBroadcastStateInternals}. This is based on the tests for
- * {@code InMemoryStateInternals}.
- */
-@RunWith(JUnit4.class)
-public class FlinkBroadcastStateInternalsTest {
-  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
-  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
-  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
-
-  private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
-      StateTags.value("stringValue", StringUtf8Coder.of());
-  private static final StateTag<Object, CombiningState<Integer, int[], Integer>>
-      SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal(
-          "sumInteger", VarIntCoder.of(), Sum.ofIntegers());
-  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
-      StateTags.bag("stringBag", StringUtf8Coder.of());
-
-  FlinkBroadcastStateInternals<String> underTest;
-
-  @Before
-  public void initStateInternals() {
-    MemoryStateBackend backend = new MemoryStateBackend();
-    try {
-      OperatorStateBackend operatorStateBackend =
-          backend.createOperatorStateBackend(new DummyEnvironment("test", 1, 0), "");
-      underTest = new FlinkBroadcastStateInternals<>(1, operatorStateBackend);
-
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Test
-  public void testValue() throws Exception {
-    ValueState<String> value = underTest.state(NAMESPACE_1, STRING_VALUE_ADDR);
-
-    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
-    assertNotEquals(
-        underTest.state(NAMESPACE_2, STRING_VALUE_ADDR),
-        value);
-
-    assertThat(value.read(), Matchers.nullValue());
-    value.write("hello");
-    assertThat(value.read(), Matchers.equalTo("hello"));
-    value.write("world");
-    assertThat(value.read(), Matchers.equalTo("world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.nullValue());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
-
-  }
-
-  @Test
-  public void testBag() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
-
-    assertThat(value.read(), Matchers.emptyIterable());
-    value.add("hello");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
-
-    value.add("world");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.emptyIterable());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
-
-  }
-
-  @Test
-  public void testBagIsEmpty() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add("hello");
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeBagIntoSource() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testMergeBagIntoNewNamespace() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag1.read(), Matchers.emptyIterable());
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testCombiningValue() throws Exception {
-    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-
-    // State instances are cached, but depend on the namespace.
-    assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR)));
-
-    assertThat(value.read(), Matchers.equalTo(0));
-    value.add(2);
-    assertThat(value.read(), Matchers.equalTo(2));
-
-    value.add(3);
-    assertThat(value.read(), Matchers.equalTo(5));
-
-    value.clear();
-    assertThat(value.read(), Matchers.equalTo(0));
-    assertEquals(underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR), value);
-  }
-
-  @Test
-  public void testCombiningIsEmpty() throws Exception {
-    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add(5);
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeCombiningValueIntoSource() throws Exception {
-    CombiningState<Integer, int[], Integer> value1 =
-        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value2 =
-        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
-
-    value1.add(5);
-    value2.add(10);
-    value1.add(6);
-
-    assertThat(value1.read(), Matchers.equalTo(11));
-    assertThat(value2.read(), Matchers.equalTo(10));
-
-    // Merging clears the old values and updates the result value.
-    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value1);
-
-    assertThat(value1.read(), Matchers.equalTo(21));
-    assertThat(value2.read(), Matchers.equalTo(0));
-  }
-
-  @Test
-  public void testMergeCombiningValueIntoNewNamespace() throws Exception {
-    CombiningState<Integer, int[], Integer> value1 =
-        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value2 =
-        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value3 =
-        underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR);
-
-    value1.add(5);
-    value2.add(10);
-    value1.add(6);
-
-    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value3);
-
-    // Merging clears the old values and updates the result value.
-    assertThat(value1.read(), Matchers.equalTo(0));
-    assertThat(value2.read(), Matchers.equalTo(0));
-    assertThat(value3.read(), Matchers.equalTo(21));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
deleted file mode 100644
index 5433d07..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.apache.beam.runners.core.StateMerging;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.api.operators.KeyContext;
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlinkKeyGroupStateInternals}. This is based on the tests for
- * {@code InMemoryStateInternals}.
- */
-@RunWith(JUnit4.class)
-public class FlinkKeyGroupStateInternalsTest {
-  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
-  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
-  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
-
-  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
-      StateTags.bag("stringBag", StringUtf8Coder.of());
-
-  FlinkKeyGroupStateInternals<String> underTest;
-  private KeyedStateBackend keyedStateBackend;
-
-  @Before
-  public void initStateInternals() {
-    try {
-      keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
-      underTest = new FlinkKeyGroupStateInternals<>(StringUtf8Coder.of(), keyedStateBackend);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private KeyedStateBackend getKeyedStateBackend(int numberOfKeyGroups,
-                                                   KeyGroupRange keyGroupRange) {
-    MemoryStateBackend backend = new MemoryStateBackend();
-    try {
-      AbstractKeyedStateBackend<ByteBuffer> keyedStateBackend = backend.createKeyedStateBackend(
-          new DummyEnvironment("test", 1, 0),
-          new JobID(),
-          "test_op",
-          new GenericTypeInfo<>(ByteBuffer.class).createSerializer(new ExecutionConfig()),
-          numberOfKeyGroups,
-          keyGroupRange,
-          new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
-      keyedStateBackend.setCurrentKey(ByteBuffer.wrap(
-          CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "1")));
-      return keyedStateBackend;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Test
-  public void testBag() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
-
-    assertThat(value.read(), Matchers.emptyIterable());
-    value.add("hello");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
-
-    value.add("world");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.emptyIterable());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
-
-  }
-
-  @Test
-  public void testBagIsEmpty() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add("hello");
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeBagIntoSource() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testMergeBagIntoNewNamespace() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag1.read(), Matchers.emptyIterable());
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testKeyGroupAndCheckpoint() throws Exception {
-    // assign to keyGroup 0
-    ByteBuffer key0 = ByteBuffer.wrap(
-        CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "11111111"));
-    // assign to keyGroup 1
-    ByteBuffer key1 = ByteBuffer.wrap(
-        CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "22222222"));
-    FlinkKeyGroupStateInternals<String> allState;
-    {
-      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
-      allState = new FlinkKeyGroupStateInternals<>(
-          StringUtf8Coder.of(), keyedStateBackend);
-      BagState<String> valueForNamespace0 = allState.state(NAMESPACE_1, STRING_BAG_ADDR);
-      BagState<String> valueForNamespace1 = allState.state(NAMESPACE_2, STRING_BAG_ADDR);
-      keyedStateBackend.setCurrentKey(key0);
-      valueForNamespace0.add("0");
-      valueForNamespace1.add("2");
-      keyedStateBackend.setCurrentKey(key1);
-      valueForNamespace0.add("1");
-      valueForNamespace1.add("3");
-      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0", "1"));
-      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2", "3"));
-    }
-
-    ClassLoader classLoader = FlinkKeyGroupStateInternalsTest.class.getClassLoader();
-
-    // 1. scale up
-    ByteArrayOutputStream out0 = new ByteArrayOutputStream();
-    allState.snapshotKeyGroupState(0, new DataOutputStream(out0));
-    DataInputStream in0 = new DataInputStream(
-        new ByteArrayInputStream(out0.toByteArray()));
-    {
-      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 0));
-      FlinkKeyGroupStateInternals<String> state0 =
-          new FlinkKeyGroupStateInternals<>(
-              StringUtf8Coder.of(), keyedStateBackend);
-      state0.restoreKeyGroupState(0, in0, classLoader);
-      BagState<String> valueForNamespace0 = state0.state(NAMESPACE_1, STRING_BAG_ADDR);
-      BagState<String> valueForNamespace1 = state0.state(NAMESPACE_2, STRING_BAG_ADDR);
-      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0"));
-      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2"));
-    }
-
-    ByteArrayOutputStream out1 = new ByteArrayOutputStream();
-    allState.snapshotKeyGroupState(1, new DataOutputStream(out1));
-    DataInputStream in1 = new DataInputStream(
-        new ByteArrayInputStream(out1.toByteArray()));
-    {
-      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(1, 1));
-      FlinkKeyGroupStateInternals<String> state1 =
-          new FlinkKeyGroupStateInternals<>(
-              StringUtf8Coder.of(), keyedStateBackend);
-      state1.restoreKeyGroupState(1, in1, classLoader);
-      BagState<String> valueForNamespace0 = state1.state(NAMESPACE_1, STRING_BAG_ADDR);
-      BagState<String> valueForNamespace1 = state1.state(NAMESPACE_2, STRING_BAG_ADDR);
-      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("1"));
-      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("3"));
-    }
-
-    // 2. scale down
-    {
-      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
-      FlinkKeyGroupStateInternals<String> newAllState = new FlinkKeyGroupStateInternals<>(
-          StringUtf8Coder.of(), keyedStateBackend);
-      in0.reset();
-      in1.reset();
-      newAllState.restoreKeyGroupState(0, in0, classLoader);
-      newAllState.restoreKeyGroupState(1, in1, classLoader);
-      BagState<String> valueForNamespace0 = newAllState.state(NAMESPACE_1, STRING_BAG_ADDR);
-      BagState<String> valueForNamespace1 = newAllState.state(NAMESPACE_2, STRING_BAG_ADDR);
-      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0", "1"));
-      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2", "3"));
-    }
-
-  }
-
-  private static class TestKeyContext implements KeyContext {
-
-    private Object key;
-
-    @Override
-    public void setCurrentKey(Object key) {
-      this.key = key;
-    }
-
-    @Override
-    public Object getCurrentKey() {
-      return key;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java
deleted file mode 100644
index 08ae0c4..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.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 org.apache.beam.runners.flink.streaming;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.state.OperatorStateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlinkSplitStateInternals}. This is based on the tests for
- * {@code InMemoryStateInternals}.
- */
-@RunWith(JUnit4.class)
-public class FlinkSplitStateInternalsTest {
-  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
-  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
-
-  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
-      StateTags.bag("stringBag", StringUtf8Coder.of());
-
-  FlinkSplitStateInternals<String> underTest;
-
-  @Before
-  public void initStateInternals() {
-    MemoryStateBackend backend = new MemoryStateBackend();
-    try {
-      OperatorStateBackend operatorStateBackend =
-          backend.createOperatorStateBackend(new DummyEnvironment("test", 1, 0), "");
-      underTest = new FlinkSplitStateInternals<>(operatorStateBackend);
-
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Test
-  public void testBag() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
-
-    assertThat(value.read(), Matchers.emptyIterable());
-    value.add("hello");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
-
-    value.add("world");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.emptyIterable());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
-
-  }
-
-  @Test
-  public void testBagIsEmpty() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add("hello");
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
deleted file mode 100644
index d140271..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
+++ /dev/null
@@ -1,395 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertThat;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.apache.beam.runners.core.StateMerging;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
-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.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.GroupingState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlinkStateInternals}. This is based on the tests for
- * {@code InMemoryStateInternals}.
- */
-@RunWith(JUnit4.class)
-public class FlinkStateInternalsTest {
-  private static final BoundedWindow WINDOW_1 = new IntervalWindow(new Instant(0), new Instant(10));
-  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
-  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
-  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
-
-  private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
-      StateTags.value("stringValue", StringUtf8Coder.of());
-  private static final StateTag<Object, CombiningState<Integer, int[], Integer>>
-      SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal(
-          "sumInteger", VarIntCoder.of(), Sum.ofIntegers());
-  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
-      StateTags.bag("stringBag", StringUtf8Coder.of());
-  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>>
-      WATERMARK_EARLIEST_ADDR =
-      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
-  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>>
-      WATERMARK_LATEST_ADDR =
-      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtLatestInputTimestamp());
-  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_EOW_ADDR =
-      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEndOfWindow());
-
-  FlinkStateInternals<String> underTest;
-
-  @Before
-  public void initStateInternals() {
-    MemoryStateBackend backend = new MemoryStateBackend();
-    try {
-      AbstractKeyedStateBackend<ByteBuffer> keyedStateBackend = backend.createKeyedStateBackend(
-          new DummyEnvironment("test", 1, 0),
-          new JobID(),
-          "test_op",
-          new GenericTypeInfo<>(ByteBuffer.class).createSerializer(new ExecutionConfig()),
-          1,
-          new KeyGroupRange(0, 0),
-          new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
-      underTest = new FlinkStateInternals<>(keyedStateBackend, StringUtf8Coder.of());
-
-      keyedStateBackend.setCurrentKey(
-          ByteBuffer.wrap(CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "Hello")));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Test
-  public void testValue() throws Exception {
-    ValueState<String> value = underTest.state(NAMESPACE_1, STRING_VALUE_ADDR);
-
-    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
-    assertNotEquals(
-        underTest.state(NAMESPACE_2, STRING_VALUE_ADDR),
-        value);
-
-    assertThat(value.read(), Matchers.nullValue());
-    value.write("hello");
-    assertThat(value.read(), Matchers.equalTo("hello"));
-    value.write("world");
-    assertThat(value.read(), Matchers.equalTo("world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.nullValue());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
-
-  }
-
-  @Test
-  public void testBag() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
-
-    assertThat(value.read(), Matchers.emptyIterable());
-    value.add("hello");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
-
-    value.add("world");
-    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
-
-    value.clear();
-    assertThat(value.read(), Matchers.emptyIterable());
-    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
-
-  }
-
-  @Test
-  public void testBagIsEmpty() throws Exception {
-    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add("hello");
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeBagIntoSource() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testMergeBagIntoNewNamespace() throws Exception {
-    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
-    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
-    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
-
-    bag1.add("Hello");
-    bag2.add("World");
-    bag1.add("!");
-
-    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
-
-    // Reading the merged bag gets both the contents
-    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
-    assertThat(bag1.read(), Matchers.emptyIterable());
-    assertThat(bag2.read(), Matchers.emptyIterable());
-  }
-
-  @Test
-  public void testCombiningValue() throws Exception {
-    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-
-    // State instances are cached, but depend on the namespace.
-    assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR)));
-
-    assertThat(value.read(), Matchers.equalTo(0));
-    value.add(2);
-    assertThat(value.read(), Matchers.equalTo(2));
-
-    value.add(3);
-    assertThat(value.read(), Matchers.equalTo(5));
-
-    value.clear();
-    assertThat(value.read(), Matchers.equalTo(0));
-    assertEquals(underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR), value);
-  }
-
-  @Test
-  public void testCombiningIsEmpty() throws Exception {
-    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add(5);
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeCombiningValueIntoSource() throws Exception {
-    CombiningState<Integer, int[], Integer> value1 =
-        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value2 =
-        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
-
-    value1.add(5);
-    value2.add(10);
-    value1.add(6);
-
-    assertThat(value1.read(), Matchers.equalTo(11));
-    assertThat(value2.read(), Matchers.equalTo(10));
-
-    // Merging clears the old values and updates the result value.
-    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value1);
-
-    assertThat(value1.read(), Matchers.equalTo(21));
-    assertThat(value2.read(), Matchers.equalTo(0));
-  }
-
-  @Test
-  public void testMergeCombiningValueIntoNewNamespace() throws Exception {
-    CombiningState<Integer, int[], Integer> value1 =
-        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value2 =
-        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
-    CombiningState<Integer, int[], Integer> value3 =
-        underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR);
-
-    value1.add(5);
-    value2.add(10);
-    value1.add(6);
-
-    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value3);
-
-    // Merging clears the old values and updates the result value.
-    assertThat(value1.read(), Matchers.equalTo(0));
-    assertThat(value2.read(), Matchers.equalTo(0));
-    assertThat(value3.read(), Matchers.equalTo(21));
-  }
-
-  @Test
-  public void testWatermarkEarliestState() throws Exception {
-    WatermarkHoldState<BoundedWindow> value =
-        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
-
-    // State instances are cached, but depend on the namespace.
-    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR)));
-
-    assertThat(value.read(), Matchers.nullValue());
-    value.add(new Instant(2000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
-
-    value.add(new Instant(3000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
-
-    value.add(new Instant(1000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(1000)));
-
-    value.clear();
-    assertThat(value.read(), Matchers.equalTo(null));
-    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR), value);
-  }
-
-  @Test
-  public void testWatermarkLatestState() throws Exception {
-    WatermarkHoldState<BoundedWindow> value =
-        underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR);
-
-    // State instances are cached, but depend on the namespace.
-    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR)));
-
-    assertThat(value.read(), Matchers.nullValue());
-    value.add(new Instant(2000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
-
-    value.add(new Instant(3000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(3000)));
-
-    value.add(new Instant(1000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(3000)));
-
-    value.clear();
-    assertThat(value.read(), Matchers.equalTo(null));
-    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR), value);
-  }
-
-  @Test
-  public void testWatermarkEndOfWindowState() throws Exception {
-    WatermarkHoldState<BoundedWindow> value = underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR);
-
-    // State instances are cached, but depend on the namespace.
-    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR));
-    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EOW_ADDR)));
-
-    assertThat(value.read(), Matchers.nullValue());
-    value.add(new Instant(2000));
-    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
-
-    value.clear();
-    assertThat(value.read(), Matchers.equalTo(null));
-    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR), value);
-  }
-
-  @Test
-  public void testWatermarkStateIsEmpty() throws Exception {
-    WatermarkHoldState<BoundedWindow> value =
-        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
-
-    assertThat(value.isEmpty().read(), Matchers.is(true));
-    ReadableState<Boolean> readFuture = value.isEmpty();
-    value.add(new Instant(1000));
-    assertThat(readFuture.read(), Matchers.is(false));
-
-    value.clear();
-    assertThat(readFuture.read(), Matchers.is(true));
-  }
-
-  @Test
-  public void testMergeEarliestWatermarkIntoSource() throws Exception {
-    WatermarkHoldState<BoundedWindow> value1 =
-        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
-    WatermarkHoldState<BoundedWindow> value2 =
-        underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR);
-
-    value1.add(new Instant(3000));
-    value2.add(new Instant(5000));
-    value1.add(new Instant(4000));
-    value2.add(new Instant(2000));
-
-    // Merging clears the old values and updates the merged value.
-    StateMerging.mergeWatermarks(Arrays.asList(value1, value2), value1, WINDOW_1);
-
-    assertThat(value1.read(), Matchers.equalTo(new Instant(2000)));
-    assertThat(value2.read(), Matchers.equalTo(null));
-  }
-
-  @Test
-  public void testMergeLatestWatermarkIntoSource() throws Exception {
-    WatermarkHoldState<BoundedWindow> value1 =
-        underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR);
-    WatermarkHoldState<BoundedWindow> value2 =
-        underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR);
-    WatermarkHoldState<BoundedWindow> value3 =
-        underTest.state(NAMESPACE_3, WATERMARK_LATEST_ADDR);
-
-    value1.add(new Instant(3000));
-    value2.add(new Instant(5000));
-    value1.add(new Instant(4000));
-    value2.add(new Instant(2000));
-
-    // Merging clears the old values and updates the result value.
-    StateMerging.mergeWatermarks(Arrays.asList(value1, value2), value3, WINDOW_1);
-
-    // Merging clears the old values and updates the result value.
-    assertThat(value3.read(), Matchers.equalTo(new Instant(5000)));
-    assertThat(value1.read(), Matchers.equalTo(null));
-    assertThat(value2.read(), Matchers.equalTo(null));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 663b910..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.streaming;
-
-import com.google.common.base.Joiner;
-import java.io.Serializable;
-import java.util.Arrays;
-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.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 org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Test for GroupByNullKey.
- */
-public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
-
-
-  protected String resultPath;
-
-  static final String[] EXPECTED_RESULT = new String[] {
-      "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
-  };
-
-  public GroupByNullKeyTest(){
-  }
-
-  @Override
-  protected void preSubmit() throws Exception {
-    resultPath = getTempDirPath("result");
-  }
-
-  @Override
-  protected void postSubmit() throws Exception {
-    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-  }
-
-  /**
-   * DoFn extracting user and timestamp.
-   */
-  private static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      KV<Integer, String> record = c.element();
-      int timestamp = record.getKey();
-      String userName = record.getValue();
-      if (userName != null) {
-        // Sets the implicit timestamp field to be used in windowing.
-        c.outputWithTimestamp(userName, new Instant(timestamp));
-      }
-    }
-  }
-
-  @Override
-  protected void testProgram() throws Exception {
-
-    Pipeline p = FlinkTestPipeline.createForStreaming();
-
-    PCollection<String> output =
-      p.apply(Create.of(Arrays.asList(
-          KV.<Integer, String>of(0, "user1"),
-          KV.<Integer, String>of(1, "user1"),
-          KV.<Integer, String>of(2, "user1"),
-          KV.<Integer, String>of(10, "user2"),
-          KV.<Integer, String>of(1, "user2"),
-          KV.<Integer, String>of(15000, "user2"),
-          KV.<Integer, String>of(12000, "user2"),
-          KV.<Integer, String>of(25000, "user3"))))
-          .apply(ParDo.of(new ExtractUserAndTimestamp()))
-          .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
-              .triggering(AfterWatermark.pastEndOfWindow())
-              .withAllowedLateness(Duration.ZERO)
-              .discardingFiredPanes())
-
-          .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
-            @ProcessElement
-            public void processElement(ProcessContext c) throws Exception {
-              String elem = c.element();
-              c.output(KV.<Void, String>of(null, elem));
-            }
-          }))
-          .apply(GroupByKey.<Void, String>create())
-          .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
-            @ProcessElement
-            public void processElement(ProcessContext c) throws Exception {
-              KV<Void, Iterable<String>> elem = c.element();
-              StringBuilder str = new StringBuilder();
-              str.append("k: " + elem.getKey() + " v:");
-              for (String v : elem.getValue()) {
-                str.append(" " + v);
-              }
-              c.output(str.toString());
-            }
-          }));
-    output.apply(TextIO.Write.to(resultPath));
-    p.run();
-  }
-}


[02/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
new file mode 100644
index 0000000..4c826d1
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java
@@ -0,0 +1,600 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.Collections;
+import java.util.HashMap;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.StatefulDoFnRunner;
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
+import org.apache.beam.sdk.coders.Coder;
+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.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PCollectionViewTesting;
+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.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link DoFnOperator}.
+ */
+@RunWith(JUnit4.class)
+public class DoFnOperatorTest {
+
+  // views and windows for testing side inputs
+  private static final long WINDOW_MSECS_1 = 100;
+  private static final long WINDOW_MSECS_2 = 500;
+
+  private WindowingStrategy<Object, IntervalWindow> windowingStrategy1 =
+      WindowingStrategy.of(FixedWindows.of(new Duration(WINDOW_MSECS_1)));
+
+  private PCollectionView<Iterable<String>> view1 =
+      PCollectionViewTesting.testingView(
+          new TupleTag<Iterable<WindowedValue<String>>>() {},
+          new PCollectionViewTesting.IdentityViewFn<String>(),
+          StringUtf8Coder.of(),
+          windowingStrategy1);
+
+  private WindowingStrategy<Object, IntervalWindow> windowingStrategy2 =
+      WindowingStrategy.of(FixedWindows.of(new Duration(WINDOW_MSECS_2)));
+
+  private PCollectionView<Iterable<String>> view2 =
+      PCollectionViewTesting.testingView(
+          new TupleTag<Iterable<WindowedValue<String>>>() {},
+          new PCollectionViewTesting.IdentityViewFn<String>(),
+          StringUtf8Coder.of(),
+          windowingStrategy2);
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testSingleOutput() throws Exception {
+
+    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
+
+    TupleTag<String> outputTag = new TupleTag<>("main-output");
+
+    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
+        new IdentityDoFn<String>(),
+        windowedValueCoder,
+        outputTag,
+        Collections.<TupleTag<?>>emptyList(),
+        new DoFnOperator.DefaultOutputManagerFactory(),
+        WindowingStrategy.globalDefault(),
+        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
+        null);
+
+    OneInputStreamOperatorTestHarness<WindowedValue<String>, String> testHarness =
+        new OneInputStreamOperatorTestHarness<>(doFnOperator);
+
+    testHarness.open();
+
+    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("Hello")));
+
+    assertThat(
+        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(WindowedValue.valueInGlobalWindow("Hello")));
+
+    testHarness.close();
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testMultiOutputOutput() throws Exception {
+
+    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
+
+    TupleTag<String> mainOutput = new TupleTag<>("main-output");
+    TupleTag<String> additionalOutput1 = new TupleTag<>("output-1");
+    TupleTag<String> additionalOutput2 = new TupleTag<>("output-2");
+    ImmutableMap<TupleTag<?>, Integer> outputMapping = ImmutableMap.<TupleTag<?>, Integer>builder()
+        .put(mainOutput, 1)
+        .put(additionalOutput1, 2)
+        .put(additionalOutput2, 3)
+        .build();
+
+    DoFnOperator<String, String, RawUnionValue> doFnOperator = new DoFnOperator<>(
+        new MultiOutputDoFn(additionalOutput1, additionalOutput2),
+        windowedValueCoder,
+        mainOutput,
+        ImmutableList.<TupleTag<?>>of(additionalOutput1, additionalOutput2),
+        new DoFnOperator.MultiOutputOutputManagerFactory(outputMapping),
+        WindowingStrategy.globalDefault(),
+        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
+        null);
+
+    OneInputStreamOperatorTestHarness<WindowedValue<String>, RawUnionValue> testHarness =
+        new OneInputStreamOperatorTestHarness<>(doFnOperator);
+
+    testHarness.open();
+
+    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("one")));
+    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("two")));
+    testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("hello")));
+
+    assertThat(
+        this.stripStreamRecordFromRawUnion(testHarness.getOutput()),
+        contains(
+            new RawUnionValue(2, WindowedValue.valueInGlobalWindow("extra: one")),
+            new RawUnionValue(3, WindowedValue.valueInGlobalWindow("extra: two")),
+            new RawUnionValue(1, WindowedValue.valueInGlobalWindow("got: hello")),
+            new RawUnionValue(2, WindowedValue.valueInGlobalWindow("got: hello")),
+            new RawUnionValue(3, WindowedValue.valueInGlobalWindow("got: hello"))));
+
+    testHarness.close();
+  }
+
+  @Test
+  public void testLateDroppingForStatefulFn() throws Exception {
+
+    WindowingStrategy<Object, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(new Duration(10)));
+
+    DoFn<Integer, String> fn = new DoFn<Integer, String>() {
+
+      @StateId("state")
+      private final StateSpec<Object, ValueState<String>> stateSpec =
+          StateSpecs.value(StringUtf8Coder.of());
+
+      @ProcessElement
+      public void processElement(ProcessContext context) {
+        context.output(context.element().toString());
+      }
+    };
+
+    WindowedValue.FullWindowedValueCoder<Integer> windowedValueCoder =
+        WindowedValue.getFullCoder(
+            VarIntCoder.of(),
+            windowingStrategy.getWindowFn().windowCoder());
+
+    TupleTag<String> outputTag = new TupleTag<>("main-output");
+
+    DoFnOperator<Integer, String, WindowedValue<String>> doFnOperator = new DoFnOperator<>(
+        fn,
+        windowedValueCoder,
+        outputTag,
+        Collections.<TupleTag<?>>emptyList(),
+        new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<String>>(),
+        windowingStrategy,
+        new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+        Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
+        VarIntCoder.of() /* key coder */);
+
+    OneInputStreamOperatorTestHarness<WindowedValue<Integer>, WindowedValue<String>> testHarness =
+        new KeyedOneInputStreamOperatorTestHarness<>(
+            doFnOperator,
+            new KeySelector<WindowedValue<Integer>, Integer>() {
+              @Override
+              public Integer getKey(WindowedValue<Integer> integerWindowedValue) throws Exception {
+                return integerWindowedValue.getValue();
+              }
+            },
+            new CoderTypeInformation<>(VarIntCoder.of()));
+
+    testHarness.open();
+
+    testHarness.processWatermark(0);
+
+    IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10));
+
+    // this should not be late
+    testHarness.processElement(
+        new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING)));
+
+    assertThat(
+        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(WindowedValue.of("13", new Instant(0), window1, PaneInfo.NO_FIRING)));
+
+    testHarness.getOutput().clear();
+
+    testHarness.processWatermark(9);
+
+    // this should still not be considered late
+    testHarness.processElement(
+        new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING)));
+
+    assertThat(
+        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(WindowedValue.of("17", new Instant(0), window1, PaneInfo.NO_FIRING)));
+
+    testHarness.getOutput().clear();
+
+    testHarness.processWatermark(10);
+
+    // this should now be considered late
+    testHarness.processElement(
+        new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING)));
+
+    assertThat(
+        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        emptyIterable());
+
+    testHarness.close();
+  }
+
+  @Test
+  public void testStateGCForStatefulFn() throws Exception {
+
+    WindowingStrategy<Object, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(new Duration(10))).withAllowedLateness(Duration.ZERO);
+
+    final String timerId = "boo";
+    final String stateId = "dazzle";
+
+    final int offset = 5000;
+    final int timerOutput = 4093;
+
+    DoFn<KV<String, Integer>, KV<String, Integer>> fn =
+        new DoFn<KV<String, Integer>, KV<String, Integer>>() {
+
+          @TimerId(timerId)
+          private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+          @StateId(stateId)
+          private final StateSpec<Object, ValueState<String>> stateSpec =
+              StateSpecs.value(StringUtf8Coder.of());
+
+          @ProcessElement
+          public void processElement(
+              ProcessContext context,
+              @TimerId(timerId) Timer timer,
+              @StateId(stateId) ValueState<String> state,
+              BoundedWindow window) {
+            timer.set(window.maxTimestamp());
+            state.write(context.element().getKey());
+            context.output(
+                KV.of(context.element().getKey(), context.element().getValue() + offset));
+          }
+
+          @OnTimer(timerId)
+          public void onTimer(OnTimerContext context, @StateId(stateId) ValueState<String> state) {
+            context.output(KV.of(state.read(), timerOutput));
+          }
+        };
+
+    WindowedValue.FullWindowedValueCoder<KV<String, Integer>> windowedValueCoder =
+        WindowedValue.getFullCoder(
+            KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
+            windowingStrategy.getWindowFn().windowCoder());
+
+    TupleTag<KV<String, Integer>> outputTag = new TupleTag<>("main-output");
+
+    DoFnOperator<
+        KV<String, Integer>, KV<String, Integer>, WindowedValue<KV<String, Integer>>> doFnOperator =
+        new DoFnOperator<>(
+            fn,
+            windowedValueCoder,
+            outputTag,
+            Collections.<TupleTag<?>>emptyList(),
+            new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<String, Integer>>>(),
+            windowingStrategy,
+            new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+            Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+            PipelineOptionsFactory.as(FlinkPipelineOptions.class),
+            StringUtf8Coder.of() /* key coder */);
+
+    KeyedOneInputStreamOperatorTestHarness<
+        String,
+        WindowedValue<KV<String, Integer>>,
+        WindowedValue<KV<String, Integer>>> testHarness =
+        new KeyedOneInputStreamOperatorTestHarness<>(
+            doFnOperator,
+            new KeySelector<WindowedValue<KV<String, Integer>>, String>() {
+              @Override
+              public String getKey(
+                  WindowedValue<KV<String, Integer>> kvWindowedValue) throws Exception {
+                return kvWindowedValue.getValue().getKey();
+              }
+            },
+            new CoderTypeInformation<>(StringUtf8Coder.of()));
+
+    testHarness.open();
+
+    testHarness.processWatermark(0);
+
+    assertEquals(0, testHarness.numKeyedStateEntries());
+
+    IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10));
+
+    testHarness.processElement(
+        new StreamRecord<>(
+            WindowedValue.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING)));
+
+    testHarness.processElement(
+        new StreamRecord<>(
+            WindowedValue.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING)));
+
+    assertThat(
+        this.<KV<String, Integer>>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(
+            WindowedValue.of(
+                KV.of("key1", 5 + offset), new Instant(1), window1, PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                KV.of("key2", 7 + offset), new Instant(3), window1, PaneInfo.NO_FIRING)));
+
+    assertEquals(2, testHarness.numKeyedStateEntries());
+
+    testHarness.getOutput().clear();
+
+    // this should trigger both the window.maxTimestamp() timer and the GC timer
+    // this tests that the GC timer fires after the user timer
+    testHarness.processWatermark(
+        window1.maxTimestamp()
+            .plus(windowingStrategy.getAllowedLateness())
+            .plus(StatefulDoFnRunner.TimeInternalsCleanupTimer.GC_DELAY_MS)
+            .getMillis());
+
+    assertThat(
+        this.<KV<String, Integer>>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(
+            WindowedValue.of(
+                KV.of("key1", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                KV.of("key2", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING)));
+
+    // ensure the state was garbage collected
+    assertEquals(0, testHarness.numKeyedStateEntries());
+
+    testHarness.close();
+  }
+
+  public void testSideInputs(boolean keyed) throws Exception {
+
+    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
+
+    TupleTag<String> outputTag = new TupleTag<>("main-output");
+
+    ImmutableMap<Integer, PCollectionView<?>> sideInputMapping =
+        ImmutableMap.<Integer, PCollectionView<?>>builder()
+            .put(1, view1)
+            .put(2, view2)
+            .build();
+
+    Coder<String> keyCoder = null;
+    if (keyed) {
+      keyCoder = StringUtf8Coder.of();
+    }
+
+    DoFnOperator<String, String, String> doFnOperator = new DoFnOperator<>(
+        new IdentityDoFn<String>(),
+        windowedValueCoder,
+        outputTag,
+        Collections.<TupleTag<?>>emptyList(),
+        new DoFnOperator.DefaultOutputManagerFactory<String>(),
+        WindowingStrategy.globalDefault(),
+        sideInputMapping, /* side-input mapping */
+        ImmutableList.<PCollectionView<?>>of(view1, view2), /* side inputs */
+        PipelineOptionsFactory.as(FlinkPipelineOptions.class),
+        keyCoder);
+
+    TwoInputStreamOperatorTestHarness<WindowedValue<String>, RawUnionValue, String> testHarness =
+        new TwoInputStreamOperatorTestHarness<>(doFnOperator);
+
+    if (keyed) {
+      // we use a dummy key for the second input since it is considered to be broadcast
+      testHarness = new KeyedTwoInputStreamOperatorTestHarness<>(
+          doFnOperator,
+          new StringKeySelector(),
+          new DummyKeySelector(),
+          BasicTypeInfo.STRING_TYPE_INFO);
+    }
+
+    testHarness.open();
+
+    IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100));
+    IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(500));
+
+    // test the keep of sideInputs events
+    testHarness.processElement2(
+        new StreamRecord<>(
+            new RawUnionValue(
+                1,
+                valuesInWindow(ImmutableList.of("hello", "ciao"), new Instant(0), firstWindow))));
+    testHarness.processElement2(
+        new StreamRecord<>(
+            new RawUnionValue(
+                2,
+                valuesInWindow(ImmutableList.of("foo", "bar"), new Instant(0), secondWindow))));
+
+    // push in a regular elements
+    WindowedValue<String> helloElement = valueInWindow("Hello", new Instant(0), firstWindow);
+    WindowedValue<String> worldElement = valueInWindow("World", new Instant(1000), firstWindow);
+    testHarness.processElement1(new StreamRecord<>(helloElement));
+    testHarness.processElement1(new StreamRecord<>(worldElement));
+
+    // test the keep of pushed-back events
+    testHarness.processElement2(
+        new StreamRecord<>(
+            new RawUnionValue(
+                1,
+                valuesInWindow(ImmutableList.of("hello", "ciao"),
+                    new Instant(1000), firstWindow))));
+    testHarness.processElement2(
+        new StreamRecord<>(
+            new RawUnionValue(
+                2,
+                valuesInWindow(ImmutableList.of("foo", "bar"), new Instant(1000), secondWindow))));
+
+    assertThat(
+        this.<String>stripStreamRecordFromWindowedValue(testHarness.getOutput()),
+        contains(helloElement, worldElement));
+
+    testHarness.close();
+
+  }
+
+  /**
+   * {@link TwoInputStreamOperatorTestHarness} support OperatorStateBackend,
+   * but don't support KeyedStateBackend. So we just test sideInput of normal ParDo.
+   */
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testNormalParDoSideInputs() throws Exception {
+    testSideInputs(false);
+  }
+
+  @Test
+  public void testKeyedSideInputs() throws Exception {
+    testSideInputs(true);
+  }
+
+  private <T> Iterable<WindowedValue<T>> stripStreamRecordFromWindowedValue(
+      Iterable<Object> input) {
+
+    return FluentIterable.from(input).filter(new Predicate<Object>() {
+      @Override
+      public boolean apply(@Nullable Object o) {
+        return o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof WindowedValue;
+      }
+    }).transform(new Function<Object, WindowedValue<T>>() {
+      @Nullable
+      @Override
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      public WindowedValue<T> apply(@Nullable Object o) {
+        if (o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof WindowedValue) {
+          return (WindowedValue) ((StreamRecord) o).getValue();
+        }
+        throw new RuntimeException("unreachable");
+      }
+    });
+  }
+
+  private Iterable<RawUnionValue> stripStreamRecordFromRawUnion(Iterable<Object> input) {
+    return FluentIterable.from(input).filter(new Predicate<Object>() {
+      @Override
+      public boolean apply(@Nullable Object o) {
+        return o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof RawUnionValue;
+      }
+    }).transform(new Function<Object, RawUnionValue>() {
+      @Nullable
+      @Override
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      public RawUnionValue apply(@Nullable Object o) {
+        if (o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof RawUnionValue) {
+          return (RawUnionValue) ((StreamRecord) o).getValue();
+        }
+        throw new RuntimeException("unreachable");
+      }
+    });
+  }
+
+  private static class MultiOutputDoFn extends DoFn<String, String> {
+    private TupleTag<String> additionalOutput1;
+    private TupleTag<String> additionalOutput2;
+
+    public MultiOutputDoFn(TupleTag<String> additionalOutput1, TupleTag<String> additionalOutput2) {
+      this.additionalOutput1 = additionalOutput1;
+      this.additionalOutput2 = additionalOutput2;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      if (c.element().equals("one")) {
+        c.output(additionalOutput1, "extra: one");
+      } else if (c.element().equals("two")) {
+        c.output(additionalOutput2, "extra: two");
+      } else {
+        c.output("got: " + c.element());
+        c.output(additionalOutput1, "got: " + c.element());
+        c.output(additionalOutput2, "got: " + c.element());
+      }
+    }
+  }
+
+  private static class IdentityDoFn<T> extends DoFn<T, T> {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(c.element());
+    }
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private WindowedValue<Iterable<?>> valuesInWindow(
+      Iterable<?> values, Instant timestamp, BoundedWindow window) {
+    return (WindowedValue) WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING);
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private <T> WindowedValue<T> valueInWindow(
+      T value, Instant timestamp, BoundedWindow window) {
+    return WindowedValue.of(value, timestamp, window, PaneInfo.NO_FIRING);
+  }
+
+
+  private static class DummyKeySelector implements KeySelector<RawUnionValue, String> {
+    @Override
+    public String getKey(RawUnionValue stringWindowedValue) throws Exception {
+      return "dummy_key";
+    }
+  }
+
+  private static class StringKeySelector implements KeySelector<WindowedValue<String>, String> {
+    @Override
+    public String getKey(WindowedValue<String> stringWindowedValue) throws Exception {
+      return stringWindowedValue.getValue();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
new file mode 100644
index 0000000..7e7d1e1
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertThat;
+
+import java.util.Arrays;
+import org.apache.beam.runners.core.StateMerging;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaceForTest;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTags;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
+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.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.GroupingState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlinkBroadcastStateInternals}. This is based on the tests for
+ * {@code InMemoryStateInternals}.
+ */
+@RunWith(JUnit4.class)
+public class FlinkBroadcastStateInternalsTest {
+  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
+  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
+  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
+
+  private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
+      StateTags.value("stringValue", StringUtf8Coder.of());
+  private static final StateTag<Object, CombiningState<Integer, int[], Integer>>
+      SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal(
+          "sumInteger", VarIntCoder.of(), Sum.ofIntegers());
+  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+      StateTags.bag("stringBag", StringUtf8Coder.of());
+
+  FlinkBroadcastStateInternals<String> underTest;
+
+  @Before
+  public void initStateInternals() {
+    MemoryStateBackend backend = new MemoryStateBackend();
+    try {
+      OperatorStateBackend operatorStateBackend =
+          backend.createOperatorStateBackend(new DummyEnvironment("test", 1, 0), "");
+      underTest = new FlinkBroadcastStateInternals<>(1, operatorStateBackend);
+
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testValue() throws Exception {
+    ValueState<String> value = underTest.state(NAMESPACE_1, STRING_VALUE_ADDR);
+
+    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
+    assertNotEquals(
+        underTest.state(NAMESPACE_2, STRING_VALUE_ADDR),
+        value);
+
+    assertThat(value.read(), Matchers.nullValue());
+    value.write("hello");
+    assertThat(value.read(), Matchers.equalTo("hello"));
+    value.write("world");
+    assertThat(value.read(), Matchers.equalTo("world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.nullValue());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
+
+  }
+
+  @Test
+  public void testBag() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
+
+    assertThat(value.read(), Matchers.emptyIterable());
+    value.add("hello");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
+
+    value.add("world");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.emptyIterable());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
+
+  }
+
+  @Test
+  public void testBagIsEmpty() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add("hello");
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeBagIntoSource() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testMergeBagIntoNewNamespace() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag1.read(), Matchers.emptyIterable());
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testCombiningValue() throws Exception {
+    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+
+    // State instances are cached, but depend on the namespace.
+    assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR)));
+
+    assertThat(value.read(), Matchers.equalTo(0));
+    value.add(2);
+    assertThat(value.read(), Matchers.equalTo(2));
+
+    value.add(3);
+    assertThat(value.read(), Matchers.equalTo(5));
+
+    value.clear();
+    assertThat(value.read(), Matchers.equalTo(0));
+    assertEquals(underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR), value);
+  }
+
+  @Test
+  public void testCombiningIsEmpty() throws Exception {
+    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add(5);
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeCombiningValueIntoSource() throws Exception {
+    CombiningState<Integer, int[], Integer> value1 =
+        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value2 =
+        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
+
+    value1.add(5);
+    value2.add(10);
+    value1.add(6);
+
+    assertThat(value1.read(), Matchers.equalTo(11));
+    assertThat(value2.read(), Matchers.equalTo(10));
+
+    // Merging clears the old values and updates the result value.
+    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value1);
+
+    assertThat(value1.read(), Matchers.equalTo(21));
+    assertThat(value2.read(), Matchers.equalTo(0));
+  }
+
+  @Test
+  public void testMergeCombiningValueIntoNewNamespace() throws Exception {
+    CombiningState<Integer, int[], Integer> value1 =
+        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value2 =
+        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value3 =
+        underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR);
+
+    value1.add(5);
+    value2.add(10);
+    value1.add(6);
+
+    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value3);
+
+    // Merging clears the old values and updates the result value.
+    assertThat(value1.read(), Matchers.equalTo(0));
+    assertThat(value2.read(), Matchers.equalTo(0));
+    assertThat(value3.read(), Matchers.equalTo(21));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
new file mode 100644
index 0000000..5433d07
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.apache.beam.runners.core.StateMerging;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaceForTest;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTags;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlinkKeyGroupStateInternals}. This is based on the tests for
+ * {@code InMemoryStateInternals}.
+ */
+@RunWith(JUnit4.class)
+public class FlinkKeyGroupStateInternalsTest {
+  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
+  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
+  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
+
+  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+      StateTags.bag("stringBag", StringUtf8Coder.of());
+
+  FlinkKeyGroupStateInternals<String> underTest;
+  private KeyedStateBackend keyedStateBackend;
+
+  @Before
+  public void initStateInternals() {
+    try {
+      keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
+      underTest = new FlinkKeyGroupStateInternals<>(StringUtf8Coder.of(), keyedStateBackend);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private KeyedStateBackend getKeyedStateBackend(int numberOfKeyGroups,
+                                                   KeyGroupRange keyGroupRange) {
+    MemoryStateBackend backend = new MemoryStateBackend();
+    try {
+      AbstractKeyedStateBackend<ByteBuffer> keyedStateBackend = backend.createKeyedStateBackend(
+          new DummyEnvironment("test", 1, 0),
+          new JobID(),
+          "test_op",
+          new GenericTypeInfo<>(ByteBuffer.class).createSerializer(new ExecutionConfig()),
+          numberOfKeyGroups,
+          keyGroupRange,
+          new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
+      keyedStateBackend.setCurrentKey(ByteBuffer.wrap(
+          CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "1")));
+      return keyedStateBackend;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testBag() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
+
+    assertThat(value.read(), Matchers.emptyIterable());
+    value.add("hello");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
+
+    value.add("world");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.emptyIterable());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
+
+  }
+
+  @Test
+  public void testBagIsEmpty() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add("hello");
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeBagIntoSource() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testMergeBagIntoNewNamespace() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag1.read(), Matchers.emptyIterable());
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testKeyGroupAndCheckpoint() throws Exception {
+    // assign to keyGroup 0
+    ByteBuffer key0 = ByteBuffer.wrap(
+        CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "11111111"));
+    // assign to keyGroup 1
+    ByteBuffer key1 = ByteBuffer.wrap(
+        CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "22222222"));
+    FlinkKeyGroupStateInternals<String> allState;
+    {
+      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
+      allState = new FlinkKeyGroupStateInternals<>(
+          StringUtf8Coder.of(), keyedStateBackend);
+      BagState<String> valueForNamespace0 = allState.state(NAMESPACE_1, STRING_BAG_ADDR);
+      BagState<String> valueForNamespace1 = allState.state(NAMESPACE_2, STRING_BAG_ADDR);
+      keyedStateBackend.setCurrentKey(key0);
+      valueForNamespace0.add("0");
+      valueForNamespace1.add("2");
+      keyedStateBackend.setCurrentKey(key1);
+      valueForNamespace0.add("1");
+      valueForNamespace1.add("3");
+      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0", "1"));
+      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2", "3"));
+    }
+
+    ClassLoader classLoader = FlinkKeyGroupStateInternalsTest.class.getClassLoader();
+
+    // 1. scale up
+    ByteArrayOutputStream out0 = new ByteArrayOutputStream();
+    allState.snapshotKeyGroupState(0, new DataOutputStream(out0));
+    DataInputStream in0 = new DataInputStream(
+        new ByteArrayInputStream(out0.toByteArray()));
+    {
+      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 0));
+      FlinkKeyGroupStateInternals<String> state0 =
+          new FlinkKeyGroupStateInternals<>(
+              StringUtf8Coder.of(), keyedStateBackend);
+      state0.restoreKeyGroupState(0, in0, classLoader);
+      BagState<String> valueForNamespace0 = state0.state(NAMESPACE_1, STRING_BAG_ADDR);
+      BagState<String> valueForNamespace1 = state0.state(NAMESPACE_2, STRING_BAG_ADDR);
+      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0"));
+      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2"));
+    }
+
+    ByteArrayOutputStream out1 = new ByteArrayOutputStream();
+    allState.snapshotKeyGroupState(1, new DataOutputStream(out1));
+    DataInputStream in1 = new DataInputStream(
+        new ByteArrayInputStream(out1.toByteArray()));
+    {
+      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(1, 1));
+      FlinkKeyGroupStateInternals<String> state1 =
+          new FlinkKeyGroupStateInternals<>(
+              StringUtf8Coder.of(), keyedStateBackend);
+      state1.restoreKeyGroupState(1, in1, classLoader);
+      BagState<String> valueForNamespace0 = state1.state(NAMESPACE_1, STRING_BAG_ADDR);
+      BagState<String> valueForNamespace1 = state1.state(NAMESPACE_2, STRING_BAG_ADDR);
+      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("1"));
+      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("3"));
+    }
+
+    // 2. scale down
+    {
+      KeyedStateBackend keyedStateBackend = getKeyedStateBackend(2, new KeyGroupRange(0, 1));
+      FlinkKeyGroupStateInternals<String> newAllState = new FlinkKeyGroupStateInternals<>(
+          StringUtf8Coder.of(), keyedStateBackend);
+      in0.reset();
+      in1.reset();
+      newAllState.restoreKeyGroupState(0, in0, classLoader);
+      newAllState.restoreKeyGroupState(1, in1, classLoader);
+      BagState<String> valueForNamespace0 = newAllState.state(NAMESPACE_1, STRING_BAG_ADDR);
+      BagState<String> valueForNamespace1 = newAllState.state(NAMESPACE_2, STRING_BAG_ADDR);
+      assertThat(valueForNamespace0.read(), Matchers.containsInAnyOrder("0", "1"));
+      assertThat(valueForNamespace1.read(), Matchers.containsInAnyOrder("2", "3"));
+    }
+
+  }
+
+  private static class TestKeyContext implements KeyContext {
+
+    private Object key;
+
+    @Override
+    public void setCurrentKey(Object key) {
+      this.key = key;
+    }
+
+    @Override
+    public Object getCurrentKey() {
+      return key;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java
new file mode 100644
index 0000000..08ae0c4
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.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 org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaceForTest;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTags;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlinkSplitStateInternals}. This is based on the tests for
+ * {@code InMemoryStateInternals}.
+ */
+@RunWith(JUnit4.class)
+public class FlinkSplitStateInternalsTest {
+  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
+  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
+
+  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+      StateTags.bag("stringBag", StringUtf8Coder.of());
+
+  FlinkSplitStateInternals<String> underTest;
+
+  @Before
+  public void initStateInternals() {
+    MemoryStateBackend backend = new MemoryStateBackend();
+    try {
+      OperatorStateBackend operatorStateBackend =
+          backend.createOperatorStateBackend(new DummyEnvironment("test", 1, 0), "");
+      underTest = new FlinkSplitStateInternals<>(operatorStateBackend);
+
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testBag() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
+
+    assertThat(value.read(), Matchers.emptyIterable());
+    value.add("hello");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
+
+    value.add("world");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.emptyIterable());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
+
+  }
+
+  @Test
+  public void testBagIsEmpty() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add("hello");
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
new file mode 100644
index 0000000..d140271
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
@@ -0,0 +1,395 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertThat;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.apache.beam.runners.core.StateMerging;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaceForTest;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTags;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
+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.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.GroupingState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlinkStateInternals}. This is based on the tests for
+ * {@code InMemoryStateInternals}.
+ */
+@RunWith(JUnit4.class)
+public class FlinkStateInternalsTest {
+  private static final BoundedWindow WINDOW_1 = new IntervalWindow(new Instant(0), new Instant(10));
+  private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1");
+  private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2");
+  private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3");
+
+  private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
+      StateTags.value("stringValue", StringUtf8Coder.of());
+  private static final StateTag<Object, CombiningState<Integer, int[], Integer>>
+      SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal(
+          "sumInteger", VarIntCoder.of(), Sum.ofIntegers());
+  private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+      StateTags.bag("stringBag", StringUtf8Coder.of());
+  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>>
+      WATERMARK_EARLIEST_ADDR =
+      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
+  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>>
+      WATERMARK_LATEST_ADDR =
+      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtLatestInputTimestamp());
+  private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_EOW_ADDR =
+      StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEndOfWindow());
+
+  FlinkStateInternals<String> underTest;
+
+  @Before
+  public void initStateInternals() {
+    MemoryStateBackend backend = new MemoryStateBackend();
+    try {
+      AbstractKeyedStateBackend<ByteBuffer> keyedStateBackend = backend.createKeyedStateBackend(
+          new DummyEnvironment("test", 1, 0),
+          new JobID(),
+          "test_op",
+          new GenericTypeInfo<>(ByteBuffer.class).createSerializer(new ExecutionConfig()),
+          1,
+          new KeyGroupRange(0, 0),
+          new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
+      underTest = new FlinkStateInternals<>(keyedStateBackend, StringUtf8Coder.of());
+
+      keyedStateBackend.setCurrentKey(
+          ByteBuffer.wrap(CoderUtils.encodeToByteArray(StringUtf8Coder.of(), "Hello")));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testValue() throws Exception {
+    ValueState<String> value = underTest.state(NAMESPACE_1, STRING_VALUE_ADDR);
+
+    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
+    assertNotEquals(
+        underTest.state(NAMESPACE_2, STRING_VALUE_ADDR),
+        value);
+
+    assertThat(value.read(), Matchers.nullValue());
+    value.write("hello");
+    assertThat(value.read(), Matchers.equalTo("hello"));
+    value.write("world");
+    assertThat(value.read(), Matchers.equalTo("world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.nullValue());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_VALUE_ADDR), value);
+
+  }
+
+  @Test
+  public void testBag() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertEquals(value, underTest.state(NAMESPACE_1, STRING_BAG_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, STRING_BAG_ADDR)));
+
+    assertThat(value.read(), Matchers.emptyIterable());
+    value.add("hello");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello"));
+
+    value.add("world");
+    assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world"));
+
+    value.clear();
+    assertThat(value.read(), Matchers.emptyIterable());
+    assertEquals(underTest.state(NAMESPACE_1, STRING_BAG_ADDR), value);
+
+  }
+
+  @Test
+  public void testBagIsEmpty() throws Exception {
+    BagState<String> value = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add("hello");
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeBagIntoSource() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testMergeBagIntoNewNamespace() throws Exception {
+    BagState<String> bag1 = underTest.state(NAMESPACE_1, STRING_BAG_ADDR);
+    BagState<String> bag2 = underTest.state(NAMESPACE_2, STRING_BAG_ADDR);
+    BagState<String> bag3 = underTest.state(NAMESPACE_3, STRING_BAG_ADDR);
+
+    bag1.add("Hello");
+    bag2.add("World");
+    bag1.add("!");
+
+    StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3);
+
+    // Reading the merged bag gets both the contents
+    assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!"));
+    assertThat(bag1.read(), Matchers.emptyIterable());
+    assertThat(bag2.read(), Matchers.emptyIterable());
+  }
+
+  @Test
+  public void testCombiningValue() throws Exception {
+    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+
+    // State instances are cached, but depend on the namespace.
+    assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR)));
+
+    assertThat(value.read(), Matchers.equalTo(0));
+    value.add(2);
+    assertThat(value.read(), Matchers.equalTo(2));
+
+    value.add(3);
+    assertThat(value.read(), Matchers.equalTo(5));
+
+    value.clear();
+    assertThat(value.read(), Matchers.equalTo(0));
+    assertEquals(underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR), value);
+  }
+
+  @Test
+  public void testCombiningIsEmpty() throws Exception {
+    GroupingState<Integer, Integer> value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add(5);
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeCombiningValueIntoSource() throws Exception {
+    CombiningState<Integer, int[], Integer> value1 =
+        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value2 =
+        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
+
+    value1.add(5);
+    value2.add(10);
+    value1.add(6);
+
+    assertThat(value1.read(), Matchers.equalTo(11));
+    assertThat(value2.read(), Matchers.equalTo(10));
+
+    // Merging clears the old values and updates the result value.
+    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value1);
+
+    assertThat(value1.read(), Matchers.equalTo(21));
+    assertThat(value2.read(), Matchers.equalTo(0));
+  }
+
+  @Test
+  public void testMergeCombiningValueIntoNewNamespace() throws Exception {
+    CombiningState<Integer, int[], Integer> value1 =
+        underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value2 =
+        underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR);
+    CombiningState<Integer, int[], Integer> value3 =
+        underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR);
+
+    value1.add(5);
+    value2.add(10);
+    value1.add(6);
+
+    StateMerging.mergeCombiningValues(Arrays.asList(value1, value2), value3);
+
+    // Merging clears the old values and updates the result value.
+    assertThat(value1.read(), Matchers.equalTo(0));
+    assertThat(value2.read(), Matchers.equalTo(0));
+    assertThat(value3.read(), Matchers.equalTo(21));
+  }
+
+  @Test
+  public void testWatermarkEarliestState() throws Exception {
+    WatermarkHoldState<BoundedWindow> value =
+        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
+
+    // State instances are cached, but depend on the namespace.
+    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR)));
+
+    assertThat(value.read(), Matchers.nullValue());
+    value.add(new Instant(2000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
+
+    value.add(new Instant(3000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
+
+    value.add(new Instant(1000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(1000)));
+
+    value.clear();
+    assertThat(value.read(), Matchers.equalTo(null));
+    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR), value);
+  }
+
+  @Test
+  public void testWatermarkLatestState() throws Exception {
+    WatermarkHoldState<BoundedWindow> value =
+        underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR);
+
+    // State instances are cached, but depend on the namespace.
+    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR)));
+
+    assertThat(value.read(), Matchers.nullValue());
+    value.add(new Instant(2000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
+
+    value.add(new Instant(3000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(3000)));
+
+    value.add(new Instant(1000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(3000)));
+
+    value.clear();
+    assertThat(value.read(), Matchers.equalTo(null));
+    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR), value);
+  }
+
+  @Test
+  public void testWatermarkEndOfWindowState() throws Exception {
+    WatermarkHoldState<BoundedWindow> value = underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR);
+
+    // State instances are cached, but depend on the namespace.
+    assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR));
+    assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EOW_ADDR)));
+
+    assertThat(value.read(), Matchers.nullValue());
+    value.add(new Instant(2000));
+    assertThat(value.read(), Matchers.equalTo(new Instant(2000)));
+
+    value.clear();
+    assertThat(value.read(), Matchers.equalTo(null));
+    assertEquals(underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR), value);
+  }
+
+  @Test
+  public void testWatermarkStateIsEmpty() throws Exception {
+    WatermarkHoldState<BoundedWindow> value =
+        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
+
+    assertThat(value.isEmpty().read(), Matchers.is(true));
+    ReadableState<Boolean> readFuture = value.isEmpty();
+    value.add(new Instant(1000));
+    assertThat(readFuture.read(), Matchers.is(false));
+
+    value.clear();
+    assertThat(readFuture.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testMergeEarliestWatermarkIntoSource() throws Exception {
+    WatermarkHoldState<BoundedWindow> value1 =
+        underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR);
+    WatermarkHoldState<BoundedWindow> value2 =
+        underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR);
+
+    value1.add(new Instant(3000));
+    value2.add(new Instant(5000));
+    value1.add(new Instant(4000));
+    value2.add(new Instant(2000));
+
+    // Merging clears the old values and updates the merged value.
+    StateMerging.mergeWatermarks(Arrays.asList(value1, value2), value1, WINDOW_1);
+
+    assertThat(value1.read(), Matchers.equalTo(new Instant(2000)));
+    assertThat(value2.read(), Matchers.equalTo(null));
+  }
+
+  @Test
+  public void testMergeLatestWatermarkIntoSource() throws Exception {
+    WatermarkHoldState<BoundedWindow> value1 =
+        underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR);
+    WatermarkHoldState<BoundedWindow> value2 =
+        underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR);
+    WatermarkHoldState<BoundedWindow> value3 =
+        underTest.state(NAMESPACE_3, WATERMARK_LATEST_ADDR);
+
+    value1.add(new Instant(3000));
+    value2.add(new Instant(5000));
+    value1.add(new Instant(4000));
+    value2.add(new Instant(2000));
+
+    // Merging clears the old values and updates the result value.
+    StateMerging.mergeWatermarks(Arrays.asList(value1, value2), value3, WINDOW_1);
+
+    // Merging clears the old values and updates the result value.
+    assertThat(value3.read(), Matchers.equalTo(new Instant(5000)));
+    assertThat(value1.read(), Matchers.equalTo(null));
+    assertThat(value2.read(), Matchers.equalTo(null));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
new file mode 100644
index 0000000..663b910
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import com.google.common.base.Joiner;
+import java.io.Serializable;
+import java.util.Arrays;
+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.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 org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Test for GroupByNullKey.
+ */
+public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
+
+
+  protected String resultPath;
+
+  static final String[] EXPECTED_RESULT = new String[] {
+      "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
+  };
+
+  public GroupByNullKeyTest(){
+  }
+
+  @Override
+  protected void preSubmit() throws Exception {
+    resultPath = getTempDirPath("result");
+  }
+
+  @Override
+  protected void postSubmit() throws Exception {
+    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+  }
+
+  /**
+   * DoFn extracting user and timestamp.
+   */
+  private static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      KV<Integer, String> record = c.element();
+      int timestamp = record.getKey();
+      String userName = record.getValue();
+      if (userName != null) {
+        // Sets the implicit timestamp field to be used in windowing.
+        c.outputWithTimestamp(userName, new Instant(timestamp));
+      }
+    }
+  }
+
+  @Override
+  protected void testProgram() throws Exception {
+
+    Pipeline p = FlinkTestPipeline.createForStreaming();
+
+    PCollection<String> output =
+      p.apply(Create.of(Arrays.asList(
+          KV.<Integer, String>of(0, "user1"),
+          KV.<Integer, String>of(1, "user1"),
+          KV.<Integer, String>of(2, "user1"),
+          KV.<Integer, String>of(10, "user2"),
+          KV.<Integer, String>of(1, "user2"),
+          KV.<Integer, String>of(15000, "user2"),
+          KV.<Integer, String>of(12000, "user2"),
+          KV.<Integer, String>of(25000, "user3"))))
+          .apply(ParDo.of(new ExtractUserAndTimestamp()))
+          .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
+              .triggering(AfterWatermark.pastEndOfWindow())
+              .withAllowedLateness(Duration.ZERO)
+              .discardingFiredPanes())
+
+          .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
+            @ProcessElement
+            public void processElement(ProcessContext c) throws Exception {
+              String elem = c.element();
+              c.output(KV.<Void, String>of(null, elem));
+            }
+          }))
+          .apply(GroupByKey.<Void, String>create())
+          .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
+            @ProcessElement
+            public void processElement(ProcessContext c) throws Exception {
+              KV<Void, Iterable<String>> elem = c.element();
+              StringBuilder str = new StringBuilder();
+              str.append("k: " + elem.getKey() + " v:");
+              for (String v : elem.getValue()) {
+                str.append(" " + v);
+              }
+              c.output(str.toString());
+            }
+          }));
+    output.apply(TextIO.Write.to(resultPath));
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java
new file mode 100644
index 0000000..3a08088
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.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 org.apache.beam.runners.flink.streaming;
+
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+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;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An unbounded source for testing the unbounded sources framework code.
+ *
+ * <p>Each split of this sources produces records of the form KV(split_id, i),
+ * where i counts up from 0.  Each record has a timestamp of i, and the watermark
+ * accurately tracks these timestamps.  The reader will occasionally return false
+ * from {@code advance}, in order to simulate a source where not all the data is
+ * available immediately.
+ */
+public class TestCountingSource
+    extends UnboundedSource<KV<Integer, Integer>, TestCountingSource.CounterMark> {
+  private static final Logger LOG = LoggerFactory.getLogger(TestCountingSource.class);
+
+  private static List<Integer> finalizeTracker;
+  private final int numMessagesPerShard;
+  private final int shardNumber;
+  private final boolean dedup;
+  private final boolean throwOnFirstSnapshot;
+  private final boolean allowSplitting;
+
+  /**
+   * We only allow an exception to be thrown from getCheckpointMark
+   * at most once. This must be static since the entire TestCountingSource
+   * instance may re-serialized when the pipeline recovers and retries.
+   */
+  private static boolean thrown = false;
+
+  public static void setFinalizeTracker(List<Integer> finalizeTracker) {
+    TestCountingSource.finalizeTracker = finalizeTracker;
+  }
+
+  public TestCountingSource(int numMessagesPerShard) {
+    this(numMessagesPerShard, 0, false, false, true);
+  }
+
+  public TestCountingSource withDedup() {
+    return new TestCountingSource(
+        numMessagesPerShard, shardNumber, true, throwOnFirstSnapshot, true);
+  }
+
+  private TestCountingSource withShardNumber(int shardNumber) {
+    return new TestCountingSource(
+        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, true);
+  }
+
+  public TestCountingSource withThrowOnFirstSnapshot(boolean throwOnFirstSnapshot) {
+    return new TestCountingSource(
+        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, true);
+  }
+
+  public TestCountingSource withoutSplitting() {
+    return new TestCountingSource(
+        numMessagesPerShard, shardNumber, dedup, throwOnFirstSnapshot, false);
+  }
+
+  private TestCountingSource(int numMessagesPerShard, int shardNumber, boolean dedup,
+                             boolean throwOnFirstSnapshot, boolean allowSplitting) {
+    this.numMessagesPerShard = numMessagesPerShard;
+    this.shardNumber = shardNumber;
+    this.dedup = dedup;
+    this.throwOnFirstSnapshot = throwOnFirstSnapshot;
+    this.allowSplitting = allowSplitting;
+  }
+
+  public int getShardNumber() {
+    return shardNumber;
+  }
+
+  @Override
+  public List<TestCountingSource> split(
+      int desiredNumSplits, PipelineOptions options) {
+    List<TestCountingSource> splits = new ArrayList<>();
+    int numSplits = allowSplitting ? desiredNumSplits : 1;
+    for (int i = 0; i < numSplits; i++) {
+      splits.add(withShardNumber(i));
+    }
+    return splits;
+  }
+
+  class CounterMark implements UnboundedSource.CheckpointMark {
+    int current;
+
+    public CounterMark(int current) {
+      this.current = current;
+    }
+
+    @Override
+    public void finalizeCheckpoint() {
+      if (finalizeTracker != null) {
+        finalizeTracker.add(current);
+      }
+    }
+  }
+
+  @Override
+  public Coder<CounterMark> getCheckpointMarkCoder() {
+    return DelegateCoder.of(
+        VarIntCoder.of(),
+        new DelegateCoder.CodingFunction<CounterMark, Integer>() {
+          @Override
+          public Integer apply(CounterMark input) {
+            return input.current;
+          }
+        },
+        new DelegateCoder.CodingFunction<Integer, CounterMark>() {
+          @Override
+          public CounterMark apply(Integer input) {
+            return new CounterMark(input);
+          }
+        });
+  }
+
+  @Override
+  public boolean requiresDeduping() {
+    return dedup;
+  }
+
+  /**
+   * Public only so that the checkpoint can be conveyed from {@link #getCheckpointMark()} to
+   * {@link TestCountingSource#createReader(PipelineOptions, CounterMark)} without cast.
+   */
+  public class CountingSourceReader extends UnboundedReader<KV<Integer, Integer>> {
+    private int current;
+
+    public CountingSourceReader(int startingPoint) {
+      this.current = startingPoint;
+    }
+
+    @Override
+    public boolean start() {
+      return advance();
+    }
+
+    @Override
+    public boolean advance() {
+      if (current >= numMessagesPerShard - 1) {
+        return false;
+      }
+      // If testing dedup, occasionally insert a duplicate value;
+      if (current >= 0 && dedup && ThreadLocalRandom.current().nextInt(5) == 0) {
+        return true;
+      }
+      current++;
+      return true;
+    }
+
+    @Override
+    public KV<Integer, Integer> getCurrent() {
+      return KV.of(shardNumber, current);
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() {
+      return new Instant(current);
+    }
+
+    @Override
+    public byte[] getCurrentRecordId() {
+      try {
+        return encodeToByteArray(KvCoder.of(VarIntCoder.of(), VarIntCoder.of()), getCurrent());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public void close() {}
+
+    @Override
+    public TestCountingSource getCurrentSource() {
+      return TestCountingSource.this;
+    }
+
+    @Override
+    public Instant getWatermark() {
+      // The watermark is a promise about future elements, and the timestamps of elements are
+      // strictly increasing for this source.
+      return new Instant(current + 1);
+    }
+
+    @Override
+    public CounterMark getCheckpointMark() {
+      if (throwOnFirstSnapshot && !thrown) {
+        thrown = true;
+        LOG.error("Throwing exception while checkpointing counter");
+        throw new RuntimeException("failed during checkpoint");
+      }
+      // The checkpoint can assume all records read, including the current, have
+      // been commited.
+      return new CounterMark(current);
+    }
+
+    @Override
+    public long getSplitBacklogBytes() {
+      return 7L;
+    }
+  }
+
+  @Override
+  public CountingSourceReader createReader(
+      PipelineOptions options, @Nullable CounterMark checkpointMark) {
+    if (checkpointMark == null) {
+      LOG.debug("creating reader");
+    } else {
+      LOG.debug("restoring reader from checkpoint with current = {}", checkpointMark.current);
+    }
+    return new CountingSourceReader(checkpointMark != null ? checkpointMark.current : -1);
+  }
+
+  @Override
+  public void validate() {}
+
+  @Override
+  public Coder<KV<Integer, Integer>> getDefaultOutputCoder() {
+    return KvCoder.of(VarIntCoder.of(), VarIntCoder.of());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
new file mode 100644
index 0000000..9e6bba8
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.common.base.Joiner;
+import java.io.Serializable;
+import java.util.Arrays;
+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 org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+
+/**
+ * Session window test.
+ */
+public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
+  protected String resultPath;
+
+  public TopWikipediaSessionsITCase(){
+  }
+
+  static final String[] EXPECTED_RESULT = new String[] {
+      "user: user1 value:3",
+      "user: user1 value:1",
+      "user: user2 value:4",
+      "user: user2 value:6",
+      "user: user3 value:7",
+      "user: user3 value:2"
+  };
+
+  @Override
+  protected void preSubmit() throws Exception {
+    resultPath = getTempDirPath("result");
+  }
+
+  @Override
+  protected void postSubmit() throws Exception {
+    compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+  }
+
+  @Override
+  protected void testProgram() throws Exception {
+
+    Pipeline p = FlinkTestPipeline.createForStreaming();
+
+    Long now = (System.currentTimeMillis() + 10000) / 1000;
+
+    PCollection<KV<String, Long>> output =
+      p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
+          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
+          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
+          ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
+          ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
+          ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
+          ("contributor_username", "user2"), new TableRow().set("timestamp", now)
+          .set("contributor_username", "user3"))))
+
+
+
+      .apply(ParDo.of(new DoFn<TableRow, String>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) throws Exception {
+          TableRow row = c.element();
+          long timestamp = (Integer) row.get("timestamp");
+          String userName = (String) row.get("contributor_username");
+          if (userName != null) {
+            // Sets the timestamp field to be used in windowing.
+            c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+          }
+        }
+      }))
+
+      .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
+
+      .apply(Count.<String>perElement());
+
+    PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
+      @ProcessElement
+      public void processElement(ProcessContext c) throws Exception {
+        KV<String, Long> el = c.element();
+        String out = "user: " + el.getKey() + " value:" + el.getValue();
+        c.output(out);
+      }
+    }));
+
+    format.apply(TextIO.Write.to(resultPath));
+
+    p.run();
+  }
+}


[07/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
new file mode 100644
index 0000000..123d5e7
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
@@ -0,0 +1,1044 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.flink;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.runners.core.ElementAndRestriction;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.core.SystemReduceFn;
+import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.KvToByteBufferKeySelector;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.SingletonKeyedWorkItem;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.SingletonKeyedWorkItemCoder;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.SplittableDoFnOperator;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.WindowDoFnOperator;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.WorkItemKeySelector;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.BoundedSourceWrapper;
+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.coders.StringUtf8Coder;
+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.transforms.Combine;
+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.join.UnionCoder;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+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.Reshuffle;
+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 org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.datastream.SplitStream;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class contains all the mappings between Beam and Flink
+ * <b>streaming</b> transformations. The {@link FlinkStreamingPipelineTranslator}
+ * traverses the Beam job and comes here to translate the encountered Beam transformations
+ * into Flink one, based on the mapping available in this class.
+ */
+class FlinkStreamingTransformTranslators {
+
+  // --------------------------------------------------------------------------------------------
+  //  Transform Translator Registry
+  // --------------------------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private static final Map<
+      Class<? extends PTransform>,
+      FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
+
+  // here you can find all the available translators.
+  static {
+    TRANSLATORS.put(Read.Bounded.class, new BoundedReadSourceTranslator());
+    TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
+    TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
+
+    TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoStreamingTranslator());
+    TRANSLATORS.put(
+        SplittableParDo.ProcessElements.class, new SplittableProcessElementsStreamingTranslator());
+    TRANSLATORS.put(
+        SplittableParDo.GBKIntoKeyedWorkItems.class, new GBKIntoKeyedWorkItemsTranslator());
+
+
+    TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator());
+    TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslator());
+    TRANSLATORS.put(
+        FlinkStreamingViewOverrides.CreateFlinkPCollectionView.class,
+        new CreateViewStreamingTranslator());
+
+    TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorStreaming());
+    TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
+    TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
+  }
+
+  public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(
+      PTransform<?, ?> transform) {
+    return TRANSLATORS.get(transform.getClass());
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Transformation Implementations
+  // --------------------------------------------------------------------------------------------
+
+  private static class TextIOWriteBoundStreamingTranslator
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound> {
+
+    private static final Logger LOG =
+        LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
+
+    @Override
+    public void translateNode(
+        TextIO.Write.Bound transform,
+        FlinkStreamingTranslationContext context) {
+      PValue input = context.getInput(transform);
+      DataStream<WindowedValue<String>> inputDataStream = context.getInputDataStream(input);
+
+      String filenamePrefix = transform.getFilenamePrefix();
+      String filenameSuffix = transform.getFilenameSuffix();
+      boolean needsValidation = transform.needsValidation();
+      int numShards = transform.getNumShards();
+      String shardNameTemplate = transform.getShardNameTemplate();
+
+      // TODO: Implement these. We need Flink support for this.
+      LOG.warn(
+          "Translation of TextIO.Write.needsValidation not yet supported. Is: {}.",
+          needsValidation);
+      LOG.warn(
+          "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
+          filenameSuffix);
+      LOG.warn(
+          "Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.",
+          shardNameTemplate);
+
+      DataStream<String> dataSink = inputDataStream
+          .flatMap(new FlatMapFunction<WindowedValue<String>, String>() {
+            @Override
+            public void flatMap(
+                WindowedValue<String> value,
+                Collector<String> out)
+                throws Exception {
+              out.collect(value.getValue());
+            }
+          });
+      DataStreamSink<String> output =
+          dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
+
+      if (numShards > 0) {
+        output.setParallelism(numShards);
+      }
+    }
+  }
+
+  private static class UnboundedReadSourceTranslator<T>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
+
+    @Override
+    public void translateNode(
+        Read.Unbounded<T> transform,
+        FlinkStreamingTranslationContext context) {
+      PCollection<T> output = context.getOutput(transform);
+
+      TypeInformation<WindowedValue<T>> outputTypeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+      DataStream<WindowedValue<T>> source;
+      try {
+        UnboundedSourceWrapper<T, ?> sourceWrapper =
+            new UnboundedSourceWrapper<>(
+                context.getPipelineOptions(),
+                transform.getSource(),
+                context.getExecutionEnvironment().getParallelism());
+        source = context
+            .getExecutionEnvironment()
+            .addSource(sourceWrapper).name(transform.getName()).returns(outputTypeInfo);
+      } catch (Exception e) {
+        throw new RuntimeException(
+            "Error while translating UnboundedSource: " + transform.getSource(), e);
+      }
+
+      context.setOutputDataStream(output, source);
+    }
+  }
+
+  private static class BoundedReadSourceTranslator<T>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Bounded<T>> {
+
+    @Override
+    public void translateNode(
+        Read.Bounded<T> transform,
+        FlinkStreamingTranslationContext context) {
+      PCollection<T> output = context.getOutput(transform);
+
+      TypeInformation<WindowedValue<T>> outputTypeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+
+      DataStream<WindowedValue<T>> source;
+      try {
+        BoundedSourceWrapper<T> sourceWrapper =
+            new BoundedSourceWrapper<>(
+                context.getPipelineOptions(),
+                transform.getSource(),
+                context.getExecutionEnvironment().getParallelism());
+        source = context
+            .getExecutionEnvironment()
+            .addSource(sourceWrapper).name(transform.getName()).returns(outputTypeInfo);
+      } catch (Exception e) {
+        throw new RuntimeException(
+            "Error while translating BoundedSource: " + transform.getSource(), e);
+      }
+
+      context.setOutputDataStream(output, source);
+    }
+  }
+
+  /**
+   * Wraps each element in a {@link RawUnionValue} with the given tag id.
+   */
+  private static class ToRawUnion<T> implements MapFunction<T, RawUnionValue> {
+    private final int intTag;
+
+    public ToRawUnion(int intTag) {
+      this.intTag = intTag;
+    }
+
+    @Override
+    public RawUnionValue map(T o) throws Exception {
+      return new RawUnionValue(intTag, o);
+    }
+  }
+
+  private static Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>>
+        transformSideInputs(
+          Collection<PCollectionView<?>> sideInputs,
+          FlinkStreamingTranslationContext context) {
+
+    // collect all side inputs
+    Map<TupleTag<?>, Integer> tagToIntMapping = new HashMap<>();
+    Map<Integer, PCollectionView<?>> intToViewMapping = new HashMap<>();
+    int count = 0;
+    for (PCollectionView<?> sideInput: sideInputs) {
+      TupleTag<?> tag = sideInput.getTagInternal();
+      intToViewMapping.put(count, sideInput);
+      tagToIntMapping.put(tag, count);
+      count++;
+      Coder<Iterable<WindowedValue<?>>> coder = sideInput.getCoderInternal();
+    }
+
+
+    List<Coder<?>> inputCoders = new ArrayList<>();
+    for (PCollectionView<?> sideInput: sideInputs) {
+      DataStream<Object> sideInputStream = context.getInputDataStream(sideInput);
+      TypeInformation<Object> tpe = sideInputStream.getType();
+      if (!(tpe instanceof CoderTypeInformation)) {
+        throw new IllegalStateException(
+            "Input Stream TypeInformation is no CoderTypeInformation.");
+      }
+
+      Coder<?> coder = ((CoderTypeInformation) tpe).getCoder();
+      inputCoders.add(coder);
+    }
+
+    UnionCoder unionCoder = UnionCoder.of(inputCoders);
+
+    CoderTypeInformation<RawUnionValue> unionTypeInformation =
+        new CoderTypeInformation<>(unionCoder);
+
+    // transform each side input to RawUnionValue and union them
+    DataStream<RawUnionValue> sideInputUnion = null;
+
+    for (PCollectionView<?> sideInput: sideInputs) {
+      TupleTag<?> tag = sideInput.getTagInternal();
+      final int intTag = tagToIntMapping.get(tag);
+      DataStream<Object> sideInputStream = context.getInputDataStream(sideInput);
+      DataStream<RawUnionValue> unionValueStream =
+          sideInputStream.map(new ToRawUnion<>(intTag)).returns(unionTypeInformation);
+
+      if (sideInputUnion == null) {
+        sideInputUnion = unionValueStream;
+      } else {
+        sideInputUnion = sideInputUnion.union(unionValueStream);
+      }
+    }
+
+    if (sideInputUnion == null) {
+      throw new IllegalStateException("No unioned side inputs, this indicates a bug.");
+    }
+
+    return new Tuple2<>(intToViewMapping, sideInputUnion);
+  }
+
+  /**
+   * Helper for translating {@link ParDo.MultiOutput} and {@link SplittableParDo.ProcessElements}.
+   */
+  static class ParDoTranslationHelper {
+
+    interface DoFnOperatorFactory<InputT, OutputT> {
+      DoFnOperator<InputT, OutputT, RawUnionValue> createDoFnOperator(
+          DoFn<InputT, OutputT> doFn,
+          List<PCollectionView<?>> sideInputs,
+          TupleTag<OutputT> mainOutputTag,
+          List<TupleTag<?>> additionalOutputTags,
+          FlinkStreamingTranslationContext context,
+          WindowingStrategy<?, ?> windowingStrategy,
+          Map<TupleTag<?>, Integer> tagsToLabels,
+          Coder<WindowedValue<InputT>> inputCoder,
+          Coder keyCoder,
+          Map<Integer, PCollectionView<?>> transformedSideInputs);
+    }
+
+    static <InputT, OutputT> void translateParDo(
+        String transformName,
+        DoFn<InputT, OutputT> doFn,
+        PCollection<InputT> input,
+        List<PCollectionView<?>> sideInputs,
+        Map<TupleTag<?>, PValue> outputs,
+        TupleTag<OutputT> mainOutputTag,
+        List<TupleTag<?>> additionalOutputTags,
+        FlinkStreamingTranslationContext context,
+        DoFnOperatorFactory<InputT, OutputT> doFnOperatorFactory) {
+
+      // we assume that the transformation does not change the windowing strategy.
+      WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
+
+      Map<TupleTag<?>, Integer> tagsToLabels =
+          transformTupleTagsToLabels(mainOutputTag, outputs);
+
+      SingleOutputStreamOperator<RawUnionValue> unionOutputStream;
+
+      Coder<WindowedValue<InputT>> inputCoder = context.getCoder(input);
+
+      DataStream<WindowedValue<InputT>> inputDataStream = context.getInputDataStream(input);
+
+      Coder keyCoder = null;
+      boolean stateful = false;
+      DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
+      if (signature.stateDeclarations().size() > 0
+          || signature.timerDeclarations().size() > 0) {
+        // Based on the fact that the signature is stateful, DoFnSignatures ensures
+        // that it is also keyed
+        keyCoder = ((KvCoder) input.getCoder()).getKeyCoder();
+        inputDataStream = inputDataStream.keyBy(new KvToByteBufferKeySelector(keyCoder));
+        stateful = true;
+      } else if (doFn instanceof SplittableParDo.ProcessFn) {
+        // we know that it is keyed on String
+        keyCoder = StringUtf8Coder.of();
+        stateful = true;
+      }
+
+      if (sideInputs.isEmpty()) {
+        DoFnOperator<InputT, OutputT, RawUnionValue> doFnOperator =
+            doFnOperatorFactory.createDoFnOperator(
+                doFn,
+                sideInputs,
+                mainOutputTag,
+                additionalOutputTags,
+                context,
+                windowingStrategy,
+                tagsToLabels,
+                inputCoder,
+                keyCoder,
+                new HashMap<Integer, PCollectionView<?>>() /* side-input mapping */);
+
+        UnionCoder outputUnionCoder = createUnionCoder(outputs);
+
+        CoderTypeInformation<RawUnionValue> outputUnionTypeInformation =
+            new CoderTypeInformation<>(outputUnionCoder);
+
+        unionOutputStream = inputDataStream
+            .transform(transformName, outputUnionTypeInformation, doFnOperator);
+
+      } else {
+        Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>> transformedSideInputs =
+            transformSideInputs(sideInputs, context);
+
+        DoFnOperator<InputT, OutputT, RawUnionValue> doFnOperator =
+            doFnOperatorFactory.createDoFnOperator(
+                doFn,
+                sideInputs,
+                mainOutputTag,
+                additionalOutputTags,
+                context,
+                windowingStrategy,
+                tagsToLabels,
+                inputCoder,
+                keyCoder,
+                transformedSideInputs.f0);
+
+        UnionCoder outputUnionCoder = createUnionCoder(outputs);
+
+        CoderTypeInformation<RawUnionValue> outputUnionTypeInformation =
+            new CoderTypeInformation<>(outputUnionCoder);
+
+        if (stateful) {
+          // we have to manually contruct the two-input transform because we're not
+          // allowed to have only one input keyed, normally.
+          KeyedStream keyedStream = (KeyedStream<?, InputT>) inputDataStream;
+          TwoInputTransformation<
+              WindowedValue<KV<?, InputT>>,
+              RawUnionValue,
+              WindowedValue<OutputT>> rawFlinkTransform = new TwoInputTransformation(
+              keyedStream.getTransformation(),
+              transformedSideInputs.f1.broadcast().getTransformation(),
+              transformName,
+              (TwoInputStreamOperator) doFnOperator,
+              outputUnionTypeInformation,
+              keyedStream.getParallelism());
+
+          rawFlinkTransform.setStateKeyType(keyedStream.getKeyType());
+          rawFlinkTransform.setStateKeySelectors(keyedStream.getKeySelector(), null);
+
+          unionOutputStream = new SingleOutputStreamOperator(
+                  keyedStream.getExecutionEnvironment(),
+                  rawFlinkTransform) {}; // we have to cheat around the ctor being protected
+
+          keyedStream.getExecutionEnvironment().addOperator(rawFlinkTransform);
+
+        } else {
+          unionOutputStream = inputDataStream
+              .connect(transformedSideInputs.f1.broadcast())
+              .transform(transformName, outputUnionTypeInformation, doFnOperator);
+        }
+      }
+
+      SplitStream<RawUnionValue> splitStream = unionOutputStream
+              .split(new OutputSelector<RawUnionValue>() {
+                @Override
+                public Iterable<String> select(RawUnionValue value) {
+                  return Collections.singletonList(Integer.toString(value.getUnionTag()));
+                }
+              });
+
+      for (Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+        final int outputTag = tagsToLabels.get(output.getKey());
+
+        TypeInformation outputTypeInfo = context.getTypeInfo((PCollection<?>) output.getValue());
+
+        @SuppressWarnings("unchecked")
+        DataStream unwrapped = splitStream.select(String.valueOf(outputTag))
+          .flatMap(new FlatMapFunction<RawUnionValue, Object>() {
+            @Override
+            public void flatMap(RawUnionValue value, Collector<Object> out) throws Exception {
+              out.collect(value.getValue());
+            }
+          }).returns(outputTypeInfo);
+
+        context.setOutputDataStream(output.getValue(), unwrapped);
+      }
+    }
+
+    private static Map<TupleTag<?>, Integer> transformTupleTagsToLabels(
+        TupleTag<?> mainTag,
+        Map<TupleTag<?>, PValue> allTaggedValues) {
+
+      Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
+      int count = 0;
+      tagToLabelMap.put(mainTag, count++);
+      for (TupleTag<?> key : allTaggedValues.keySet()) {
+        if (!tagToLabelMap.containsKey(key)) {
+          tagToLabelMap.put(key, count++);
+        }
+      }
+      return tagToLabelMap;
+    }
+
+    private static UnionCoder createUnionCoder(Map<TupleTag<?>, PValue> taggedCollections) {
+      List<Coder<?>> outputCoders = Lists.newArrayList();
+      for (PValue taggedColl : taggedCollections.values()) {
+        checkArgument(
+            taggedColl instanceof PCollection,
+            "A Union Coder can only be created for a Collection of Tagged %s. Got %s",
+            PCollection.class.getSimpleName(),
+            taggedColl.getClass().getSimpleName());
+        PCollection<?> coll = (PCollection<?>) taggedColl;
+        WindowedValue.FullWindowedValueCoder<?> windowedValueCoder =
+            WindowedValue.getFullCoder(
+                coll.getCoder(),
+                coll.getWindowingStrategy().getWindowFn().windowCoder());
+        outputCoders.add(windowedValueCoder);
+      }
+      return UnionCoder.of(outputCoders);
+    }
+  }
+
+  private static class ParDoStreamingTranslator<InputT, OutputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      ParDo.MultiOutput<InputT, OutputT>> {
+
+    @Override
+    public void translateNode(
+        ParDo.MultiOutput<InputT, OutputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      ParDoTranslationHelper.translateParDo(
+          transform.getName(),
+          transform.getFn(),
+          (PCollection<InputT>) context.getInput(transform),
+          transform.getSideInputs(),
+          context.getOutputs(transform),
+          transform.getMainOutputTag(),
+          transform.getAdditionalOutputTags().getAll(),
+          context,
+          new ParDoTranslationHelper.DoFnOperatorFactory<InputT, OutputT>() {
+            @Override
+            public DoFnOperator<InputT, OutputT, RawUnionValue> createDoFnOperator(
+                DoFn<InputT, OutputT> doFn,
+                List<PCollectionView<?>> sideInputs,
+                TupleTag<OutputT> mainOutputTag,
+                List<TupleTag<?>> additionalOutputTags,
+                FlinkStreamingTranslationContext context,
+                WindowingStrategy<?, ?> windowingStrategy,
+                Map<TupleTag<?>, Integer> tagsToLabels,
+                Coder<WindowedValue<InputT>> inputCoder,
+                Coder keyCoder,
+                Map<Integer, PCollectionView<?>> transformedSideInputs) {
+              return new DoFnOperator<>(
+                  doFn,
+                  inputCoder,
+                  mainOutputTag,
+                  additionalOutputTags,
+                  new DoFnOperator.MultiOutputOutputManagerFactory(tagsToLabels),
+                  windowingStrategy,
+                  transformedSideInputs,
+                  sideInputs,
+                  context.getPipelineOptions(),
+                  keyCoder);
+            }
+          });
+    }
+  }
+
+  private static class SplittableProcessElementsStreamingTranslator<
+      InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT, TrackerT>> {
+
+    @Override
+    public void translateNode(
+        SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT, TrackerT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      ParDoTranslationHelper.translateParDo(
+          transform.getName(),
+          transform.newProcessFn(transform.getFn()),
+          (PCollection<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>)
+              context.getInput(transform),
+          transform.getSideInputs(),
+          context.getOutputs(transform),
+          transform.getMainOutputTag(),
+          transform.getAdditionalOutputTags().getAll(),
+          context,
+          new ParDoTranslationHelper.DoFnOperatorFactory<
+              KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT>() {
+            @Override
+            public DoFnOperator<
+                KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>,
+                OutputT,
+                RawUnionValue> createDoFnOperator(
+                    DoFn<
+                        KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>,
+                        OutputT> doFn,
+                    List<PCollectionView<?>> sideInputs,
+                    TupleTag<OutputT> mainOutputTag,
+                    List<TupleTag<?>> additionalOutputTags,
+                    FlinkStreamingTranslationContext context,
+                    WindowingStrategy<?, ?> windowingStrategy,
+                    Map<TupleTag<?>, Integer> tagsToLabels,
+                    Coder<
+                        WindowedValue<
+                            KeyedWorkItem<
+                                String,
+                                ElementAndRestriction<InputT, RestrictionT>>>> inputCoder,
+                    Coder keyCoder,
+                    Map<Integer, PCollectionView<?>> transformedSideInputs) {
+              return new SplittableDoFnOperator<>(
+                  doFn,
+                  inputCoder,
+                  mainOutputTag,
+                  additionalOutputTags,
+                  new DoFnOperator.MultiOutputOutputManagerFactory(tagsToLabels),
+                  windowingStrategy,
+                  transformedSideInputs,
+                  sideInputs,
+                  context.getPipelineOptions(),
+                  keyCoder);
+            }
+          });
+    }
+  }
+
+  private static class CreateViewStreamingTranslator<ElemT, ViewT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      FlinkStreamingViewOverrides.CreateFlinkPCollectionView<ElemT, ViewT>> {
+
+    @Override
+    public void translateNode(
+        FlinkStreamingViewOverrides.CreateFlinkPCollectionView<ElemT, ViewT> transform,
+        FlinkStreamingTranslationContext context) {
+      // just forward
+      DataStream<WindowedValue<List<ElemT>>> inputDataSet =
+          context.getInputDataStream(context.getInput(transform));
+
+      PCollectionView<ViewT> view = context.getOutput(transform);
+
+      context.setOutputDataStream(view, inputDataSet);
+    }
+  }
+
+  private static class WindowAssignTranslator<T>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Assign<T>> {
+
+    @Override
+    public void translateNode(
+        Window.Assign<T> transform,
+        FlinkStreamingTranslationContext context) {
+
+      @SuppressWarnings("unchecked")
+      WindowingStrategy<T, BoundedWindow> windowingStrategy =
+          (WindowingStrategy<T, BoundedWindow>)
+              context.getOutput(transform).getWindowingStrategy();
+
+      TypeInformation<WindowedValue<T>> typeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+      DataStream<WindowedValue<T>> inputDataStream =
+          context.getInputDataStream(context.getInput(transform));
+
+      WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
+
+      FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
+          new FlinkAssignWindows<>(windowFn);
+
+      SingleOutputStreamOperator<WindowedValue<T>> outputDataStream = inputDataStream
+          .flatMap(assignWindowsFunction)
+          .name(context.getOutput(transform).getName())
+          .returns(typeInfo);
+
+      context.setOutputDataStream(context.getOutput(transform), outputDataStream);
+    }
+  }
+
+  private static class ReshuffleTranslatorStreaming<K, InputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<Reshuffle<K, InputT>> {
+
+    @Override
+    public void translateNode(
+        Reshuffle<K, InputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      DataStream<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataStream(context.getInput(transform));
+
+      context.setOutputDataStream(context.getOutput(transform), inputDataSet.rebalance());
+
+    }
+  }
+
+
+  private static class GroupByKeyTranslator<K, InputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, InputT>> {
+
+    @Override
+    public void translateNode(
+        GroupByKey<K, InputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      PCollection<KV<K, InputT>> input = context.getInput(transform);
+
+      @SuppressWarnings("unchecked")
+      WindowingStrategy<?, BoundedWindow> windowingStrategy =
+          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
+
+      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
+
+      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
+          inputKvCoder.getKeyCoder(),
+          inputKvCoder.getValueCoder(),
+          input.getWindowingStrategy().getWindowFn().windowCoder());
+
+      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
+
+      WindowedValue.
+          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
+          WindowedValue.getFullCoder(
+              workItemCoder,
+              input.getWindowingStrategy().getWindowFn().windowCoder());
+
+      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
+          new CoderTypeInformation<>(windowedWorkItemCoder);
+
+      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
+          inputDataStream
+              .flatMap(new ToKeyedWorkItem<K, InputT>())
+              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
+
+      KeyedStream<
+          WindowedValue<
+              SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
+          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
+
+      SystemReduceFn<K, InputT, Iterable<InputT>, Iterable<InputT>, BoundedWindow> reduceFn =
+          SystemReduceFn.buffering(inputKvCoder.getValueCoder());
+
+      TypeInformation<WindowedValue<KV<K, Iterable<InputT>>>> outputTypeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+      DoFnOperator.DefaultOutputManagerFactory<
+            WindowedValue<KV<K, Iterable<InputT>>>> outputManagerFactory =
+          new DoFnOperator.DefaultOutputManagerFactory<>();
+
+      WindowDoFnOperator<K, InputT, Iterable<InputT>> doFnOperator =
+          new WindowDoFnOperator<>(
+              reduceFn,
+              (Coder) windowedWorkItemCoder,
+              new TupleTag<KV<K, Iterable<InputT>>>("main output"),
+              Collections.<TupleTag<?>>emptyList(),
+              outputManagerFactory,
+              windowingStrategy,
+              new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+              Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+              context.getPipelineOptions(),
+              inputKvCoder.getKeyCoder());
+
+      // our operator excepts WindowedValue<KeyedWorkItem> while our input stream
+      // is WindowedValue<SingletonKeyedWorkItem>, which is fine but Java doesn't like it ...
+      @SuppressWarnings("unchecked")
+      SingleOutputStreamOperator<WindowedValue<KV<K, Iterable<InputT>>>> outDataStream =
+          keyedWorkItemStream
+              .transform(
+                  transform.getName(),
+                  outputTypeInfo,
+                  (OneInputStreamOperator) doFnOperator);
+
+      context.setOutputDataStream(context.getOutput(transform), outDataStream);
+
+    }
+  }
+
+  private static class CombinePerKeyTranslator<K, InputT, OutputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      Combine.PerKey<K, InputT, OutputT>> {
+
+    @Override
+    boolean canTranslate(
+        Combine.PerKey<K, InputT, OutputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      // if we have a merging window strategy and side inputs we cannot
+      // translate as a proper combine. We have to group and then run the combine
+      // over the final grouped values.
+      PCollection<KV<K, InputT>> input = context.getInput(transform);
+
+      @SuppressWarnings("unchecked")
+      WindowingStrategy<?, BoundedWindow> windowingStrategy =
+          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
+
+      return windowingStrategy.getWindowFn().isNonMerging() || transform.getSideInputs().isEmpty();
+    }
+
+    @Override
+    public void translateNode(
+        Combine.PerKey<K, InputT, OutputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      PCollection<KV<K, InputT>> input = context.getInput(transform);
+
+      @SuppressWarnings("unchecked")
+      WindowingStrategy<?, BoundedWindow> windowingStrategy =
+          (WindowingStrategy<?, BoundedWindow>) input.getWindowingStrategy();
+
+      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
+
+      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
+          inputKvCoder.getKeyCoder(),
+          inputKvCoder.getValueCoder(),
+          input.getWindowingStrategy().getWindowFn().windowCoder());
+
+      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
+
+      WindowedValue.
+          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
+            WindowedValue.getFullCoder(
+                workItemCoder,
+                input.getWindowingStrategy().getWindowFn().windowCoder());
+
+      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
+          new CoderTypeInformation<>(windowedWorkItemCoder);
+
+      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
+          inputDataStream
+              .flatMap(new ToKeyedWorkItem<K, InputT>())
+              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
+
+      KeyedStream<
+            WindowedValue<
+                SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
+          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
+
+      SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> reduceFn = SystemReduceFn.combining(
+          inputKvCoder.getKeyCoder(),
+          AppliedCombineFn.withInputCoder(
+              transform.getFn(), input.getPipeline().getCoderRegistry(), inputKvCoder));
+
+      TypeInformation<WindowedValue<KV<K, OutputT>>> outputTypeInfo =
+          context.getTypeInfo(context.getOutput(transform));
+
+      List<PCollectionView<?>> sideInputs = transform.getSideInputs();
+
+      if (sideInputs.isEmpty()) {
+
+        WindowDoFnOperator<K, InputT, OutputT> doFnOperator =
+            new WindowDoFnOperator<>(
+                reduceFn,
+                (Coder) windowedWorkItemCoder,
+                new TupleTag<KV<K, OutputT>>("main output"),
+                Collections.<TupleTag<?>>emptyList(),
+                new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<K, OutputT>>>(),
+                windowingStrategy,
+                new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
+                Collections.<PCollectionView<?>>emptyList(), /* side inputs */
+                context.getPipelineOptions(),
+                inputKvCoder.getKeyCoder());
+
+        // our operator excepts WindowedValue<KeyedWorkItem> while our input stream
+        // is WindowedValue<SingletonKeyedWorkItem>, which is fine but Java doesn't like it ...
+        @SuppressWarnings("unchecked")
+        SingleOutputStreamOperator<WindowedValue<KV<K, OutputT>>> outDataStream =
+            keyedWorkItemStream.transform(
+                transform.getName(), outputTypeInfo, (OneInputStreamOperator) doFnOperator);
+
+        context.setOutputDataStream(context.getOutput(transform), outDataStream);
+      } else {
+        Tuple2<Map<Integer, PCollectionView<?>>, DataStream<RawUnionValue>> transformSideInputs =
+            transformSideInputs(sideInputs, context);
+
+        WindowDoFnOperator<K, InputT, OutputT> doFnOperator =
+            new WindowDoFnOperator<>(
+                reduceFn,
+                (Coder) windowedWorkItemCoder,
+                new TupleTag<KV<K, OutputT>>("main output"),
+                Collections.<TupleTag<?>>emptyList(),
+                new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<KV<K, OutputT>>>(),
+                windowingStrategy,
+                transformSideInputs.f0,
+                sideInputs,
+                context.getPipelineOptions(),
+                inputKvCoder.getKeyCoder());
+
+        // we have to manually contruct the two-input transform because we're not
+        // allowed to have only one input keyed, normally.
+
+        TwoInputTransformation<
+            WindowedValue<SingletonKeyedWorkItem<K, InputT>>,
+            RawUnionValue,
+            WindowedValue<KV<K, OutputT>>> rawFlinkTransform = new TwoInputTransformation<>(
+            keyedWorkItemStream.getTransformation(),
+            transformSideInputs.f1.broadcast().getTransformation(),
+            transform.getName(),
+            (TwoInputStreamOperator) doFnOperator,
+            outputTypeInfo,
+            keyedWorkItemStream.getParallelism());
+
+        rawFlinkTransform.setStateKeyType(keyedWorkItemStream.getKeyType());
+        rawFlinkTransform.setStateKeySelectors(keyedWorkItemStream.getKeySelector(), null);
+
+        @SuppressWarnings({ "unchecked", "rawtypes" })
+        SingleOutputStreamOperator<WindowedValue<KV<K, OutputT>>> outDataStream =
+            new SingleOutputStreamOperator(
+                keyedWorkItemStream.getExecutionEnvironment(),
+                rawFlinkTransform) {}; // we have to cheat around the ctor being protected
+
+        keyedWorkItemStream.getExecutionEnvironment().addOperator(rawFlinkTransform);
+
+        context.setOutputDataStream(context.getOutput(transform), outDataStream);
+      }
+    }
+  }
+
+  private static class GBKIntoKeyedWorkItemsTranslator<K, InputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      SplittableParDo.GBKIntoKeyedWorkItems<K, InputT>> {
+
+    @Override
+    boolean canTranslate(
+        SplittableParDo.GBKIntoKeyedWorkItems<K, InputT> transform,
+        FlinkStreamingTranslationContext context) {
+      return true;
+    }
+
+    @Override
+    public void translateNode(
+        SplittableParDo.GBKIntoKeyedWorkItems<K, InputT> transform,
+        FlinkStreamingTranslationContext context) {
+
+      PCollection<KV<K, InputT>> input = context.getInput(transform);
+
+      KvCoder<K, InputT> inputKvCoder = (KvCoder<K, InputT>) input.getCoder();
+
+      SingletonKeyedWorkItemCoder<K, InputT> workItemCoder = SingletonKeyedWorkItemCoder.of(
+          inputKvCoder.getKeyCoder(),
+          inputKvCoder.getValueCoder(),
+          input.getWindowingStrategy().getWindowFn().windowCoder());
+
+
+      WindowedValue.
+          FullWindowedValueCoder<SingletonKeyedWorkItem<K, InputT>> windowedWorkItemCoder =
+          WindowedValue.getFullCoder(
+              workItemCoder,
+              input.getWindowingStrategy().getWindowFn().windowCoder());
+
+      CoderTypeInformation<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemTypeInfo =
+          new CoderTypeInformation<>(windowedWorkItemCoder);
+
+      DataStream<WindowedValue<KV<K, InputT>>> inputDataStream = context.getInputDataStream(input);
+
+      DataStream<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> workItemStream =
+          inputDataStream
+              .flatMap(new ToKeyedWorkItem<K, InputT>())
+              .returns(workItemTypeInfo).name("ToKeyedWorkItem");
+
+      KeyedStream<
+          WindowedValue<
+              SingletonKeyedWorkItem<K, InputT>>, ByteBuffer> keyedWorkItemStream = workItemStream
+          .keyBy(new WorkItemKeySelector<K, InputT>(inputKvCoder.getKeyCoder()));
+
+      context.setOutputDataStream(context.getOutput(transform), keyedWorkItemStream);
+    }
+  }
+
+  private static class FlattenPCollectionTranslator<T>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+      Flatten.PCollections<T>> {
+
+    @Override
+    public void translateNode(
+        Flatten.PCollections<T> transform,
+        FlinkStreamingTranslationContext context) {
+      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
+
+      if (allInputs.isEmpty()) {
+
+        // create an empty dummy source to satisfy downstream operations
+        // we cannot create an empty source in Flink, therefore we have to
+        // add the flatMap that simply never forwards the single element
+        DataStreamSource<String> dummySource =
+            context.getExecutionEnvironment().fromElements("dummy");
+
+        DataStream<WindowedValue<T>> result = dummySource.flatMap(
+            new FlatMapFunction<String, WindowedValue<T>>() {
+              @Override
+              public void flatMap(
+                  String s,
+                  Collector<WindowedValue<T>> collector) throws Exception {
+                // never return anything
+              }
+            }).returns(
+            new CoderTypeInformation<>(
+                WindowedValue.getFullCoder(
+                    (Coder<T>) VoidCoder.of(),
+                    GlobalWindow.Coder.INSTANCE)));
+        context.setOutputDataStream(context.getOutput(transform), result);
+
+      } else {
+        DataStream<T> result = null;
+        for (PValue input : allInputs.values()) {
+          DataStream<T> current = context.getInputDataStream(input);
+          result = (result == null) ? current : result.union(current);
+        }
+        context.setOutputDataStream(context.getOutput(transform), result);
+      }
+    }
+  }
+
+  private static class ToKeyedWorkItem<K, InputT>
+      extends RichFlatMapFunction<
+      WindowedValue<KV<K, InputT>>,
+      WindowedValue<SingletonKeyedWorkItem<K, InputT>>> {
+
+    @Override
+    public void flatMap(
+        WindowedValue<KV<K, InputT>> inWithMultipleWindows,
+        Collector<WindowedValue<SingletonKeyedWorkItem<K, InputT>>> out) throws Exception {
+
+      // we need to wrap each one work item per window for now
+      // since otherwise the PushbackSideInputRunner will not correctly
+      // determine whether side inputs are ready
+      //
+      // this is tracked as https://issues.apache.org/jira/browse/BEAM-1850
+      for (WindowedValue<KV<K, InputT>> in : inWithMultipleWindows.explodeWindows()) {
+        SingletonKeyedWorkItem<K, InputT> workItem =
+            new SingletonKeyedWorkItem<>(
+                in.getValue().getKey(),
+                in.withValue(in.getValue().getValue()));
+
+        out.collect(in.withValue(workItem));
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
new file mode 100644
index 0000000..1a943a3
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Iterables;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.sdk.coders.Coder;
+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.util.WindowedValue;
+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.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * Helper for keeping track of which {@link DataStream DataStreams} map
+ * to which {@link PTransform PTransforms}.
+ */
+class FlinkStreamingTranslationContext {
+
+  private final StreamExecutionEnvironment env;
+  private final PipelineOptions options;
+
+  /**
+   * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
+   * Flink Operator that produced it, after the translation of the correspondinf PTransform
+   * to its Flink equivalent.
+   * */
+  private final Map<PValue, DataStream<?>> dataStreams;
+
+  private AppliedPTransform<?, ?, ?> currentTransform;
+
+  public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
+    this.env = checkNotNull(env);
+    this.options = checkNotNull(options);
+    this.dataStreams = new HashMap<>();
+  }
+
+  public StreamExecutionEnvironment getExecutionEnvironment() {
+    return env;
+  }
+
+  public PipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> DataStream<T> getInputDataStream(PValue value) {
+    return (DataStream<T>) dataStreams.get(value);
+  }
+
+  public void setOutputDataStream(PValue value, DataStream<?> set) {
+    if (!dataStreams.containsKey(value)) {
+      dataStreams.put(value, set);
+    }
+  }
+
+  /**
+   * Sets the AppliedPTransform which carries input/output.
+   * @param currentTransform
+   */
+  public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+    this.currentTransform = currentTransform;
+  }
+
+  public <T> Coder<WindowedValue<T>> getCoder(PCollection<T> collection) {
+    Coder<T> valueCoder = collection.getCoder();
+
+    return WindowedValue.getFullCoder(
+        valueCoder,
+        collection.getWindowingStrategy().getWindowFn().windowCoder());
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
+    Coder<T> valueCoder = collection.getCoder();
+    WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
+        WindowedValue.getFullCoder(
+            valueCoder,
+            collection.getWindowingStrategy().getWindowFn().windowCoder());
+
+    return new CoderTypeInformation<>(windowedValueCoder);
+  }
+
+
+  @SuppressWarnings("unchecked")
+  public <T extends PValue> T getInput(PTransform<T, ?> transform) {
+    return (T) Iterables.getOnlyElement(currentTransform.getInputs().values());
+  }
+
+  public <T extends PInput> Map<TupleTag<?>, PValue> getInputs(PTransform<T, ?> transform) {
+    return currentTransform.getInputs();
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T extends PValue> T getOutput(PTransform<?, T> transform) {
+    return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values());
+  }
+
+  public <OutputT extends POutput> Map<TupleTag<?>, PValue> getOutputs(
+      PTransform<?, OutputT> transform) {
+    return currentTransform.getOutputs();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
new file mode 100644
index 0000000..f955f2a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+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.coders.ListCoder;
+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.View;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+/**
+ * Flink streaming overrides for various view (side input) transforms.
+ */
+class FlinkStreamingViewOverrides {
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap}
+   * for the Flink runner in streaming mode.
+   */
+  static class StreamingViewAsMap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
+
+    private final transient FlinkRunner runner;
+
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingViewAsMap(FlinkRunner runner, View.AsMap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, V>> expand(PCollection<KV<K, V>> input) {
+      PCollectionView<Map<K, V>> view =
+          PCollectionViews.mapView(
+              input,
+              input.getWindowingStrategy(),
+              input.getCoder());
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        inputCoder.getKeyCoder().verifyDeterministic();
+      } catch (Coder.NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+      }
+
+      return input
+          .apply(Combine.globally(new Concatenate<KV<K, V>>()).withoutDefaults())
+          .apply(CreateFlinkPCollectionView.<KV<K, V>, Map<K, V>>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingViewAsMap";
+    }
+  }
+
+  /**
+   * Specialized expansion for {@link
+   * View.AsMultimap View.AsMultimap} for the
+   * Flink runner in streaming mode.
+   */
+  static class StreamingViewAsMultimap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
+
+    private final transient FlinkRunner runner;
+
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, Iterable<V>>> expand(PCollection<KV<K, V>> input) {
+      PCollectionView<Map<K, Iterable<V>>> view =
+          PCollectionViews.multimapView(
+              input,
+              input.getWindowingStrategy(),
+              input.getCoder());
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        inputCoder.getKeyCoder().verifyDeterministic();
+      } catch (Coder.NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+      }
+
+      return input
+          .apply(Combine.globally(new Concatenate<KV<K, V>>()).withoutDefaults())
+          .apply(CreateFlinkPCollectionView.<KV<K, V>, Map<K, Iterable<V>>>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingViewAsMultimap";
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link View.AsList View.AsList} for the
+   * Flink runner in streaming mode.
+   */
+  static class StreamingViewAsList<T>
+      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingViewAsList(FlinkRunner runner, View.AsList<T> transform) {}
+
+    @Override
+    public PCollectionView<List<T>> expand(PCollection<T> input) {
+      PCollectionView<List<T>> view =
+          PCollectionViews.listView(
+              input,
+              input.getWindowingStrategy(),
+              input.getCoder());
+
+      return input.apply(Combine.globally(new Concatenate<T>()).withoutDefaults())
+          .apply(CreateFlinkPCollectionView.<T, List<T>>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingViewAsList";
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link View.AsIterable View.AsIterable} for the
+   * Flink runner in streaming mode.
+   */
+  static class StreamingViewAsIterable<T>
+      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable<T> transform) { }
+
+    @Override
+    public PCollectionView<Iterable<T>> expand(PCollection<T> input) {
+      PCollectionView<Iterable<T>> view =
+          PCollectionViews.iterableView(
+              input,
+              input.getWindowingStrategy(),
+              input.getCoder());
+
+      return input.apply(Combine.globally(new Concatenate<T>()).withoutDefaults())
+          .apply(CreateFlinkPCollectionView.<T, Iterable<T>>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingViewAsIterable";
+    }
+  }
+
+  /**
+   * Specialized expansion for
+   * {@link View.AsSingleton View.AsSingleton} for the
+   * Flink runner in streaming mode.
+   */
+  static class StreamingViewAsSingleton<T>
+      extends PTransform<PCollection<T>, PCollectionView<T>> {
+    private View.AsSingleton<T> transform;
+
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton<T> transform) {
+      this.transform = transform;
+    }
+
+    @Override
+    public PCollectionView<T> expand(PCollection<T> input) {
+      Combine.Globally<T, T> combine = Combine.globally(
+          new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue()));
+      if (!transform.hasDefaultValue()) {
+        combine = combine.withoutDefaults();
+      }
+      return input.apply(combine.asSingletonView());
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingViewAsSingleton";
+    }
+
+    private static class SingletonCombine<T> extends Combine.BinaryCombineFn<T> {
+      private boolean hasDefaultValue;
+      private T defaultValue;
+
+      SingletonCombine(boolean hasDefaultValue, T defaultValue) {
+        this.hasDefaultValue = hasDefaultValue;
+        this.defaultValue = defaultValue;
+      }
+
+      @Override
+      public T apply(T left, T right) {
+        throw new IllegalArgumentException("PCollection with more than one element "
+            + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to "
+            + "combine the PCollection into a single value");
+      }
+
+      @Override
+      public T identity() {
+        if (hasDefaultValue) {
+          return defaultValue;
+        } else {
+          throw new IllegalArgumentException(
+              "Empty PCollection accessed as a singleton view. "
+                  + "Consider setting withDefault to provide a default value");
+        }
+      }
+    }
+  }
+
+  static class StreamingCombineGloballyAsSingletonView<InputT, OutputT>
+      extends PTransform<PCollection<InputT>, PCollectionView<OutputT>> {
+    Combine.GloballyAsSingletonView<InputT, OutputT> transform;
+
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
+    public StreamingCombineGloballyAsSingletonView(
+        FlinkRunner runner,
+        Combine.GloballyAsSingletonView<InputT, OutputT> transform) {
+      this.transform = transform;
+    }
+
+    @Override
+    public PCollectionView<OutputT> expand(PCollection<InputT> input) {
+      PCollection<OutputT> combined =
+          input.apply(Combine.globally(transform.getCombineFn())
+              .withoutDefaults()
+              .withFanout(transform.getFanout()));
+
+      PCollectionView<OutputT> view = PCollectionViews.singletonView(
+          combined,
+          combined.getWindowingStrategy(),
+          transform.getInsertDefault(),
+          transform.getInsertDefault()
+              ? transform.getCombineFn().defaultValue() : null,
+          combined.getCoder());
+      return combined
+          .apply(ParDo.of(new WrapAsList<OutputT>()))
+          .apply(CreateFlinkPCollectionView.<OutputT, OutputT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "StreamingCombineGloballyAsSingletonView";
+    }
+  }
+
+  private static class WrapAsList<T> extends DoFn<T, List<T>> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(Collections.singletonList(c.element()));
+    }
+  }
+
+  /**
+   * Combiner that combines {@code T}s into a single {@code List<T>} containing all inputs.
+   *
+   * <p>For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap},
+   * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}.
+   * They require the input {@link PCollection} fits in memory.
+   * For a large {@link PCollection} this is expected to crash!
+   *
+   * @param <T> the type of elements to concatenate.
+   */
+  private static class Concatenate<T> extends Combine.CombineFn<T, List<T>, List<T>> {
+    @Override
+    public List<T> createAccumulator() {
+      return new ArrayList<T>();
+    }
+
+    @Override
+    public List<T> addInput(List<T> accumulator, T input) {
+      accumulator.add(input);
+      return accumulator;
+    }
+
+    @Override
+    public List<T> mergeAccumulators(Iterable<List<T>> accumulators) {
+      List<T> result = createAccumulator();
+      for (List<T> accumulator : accumulators) {
+        result.addAll(accumulator);
+      }
+      return result;
+    }
+
+    @Override
+    public List<T> extractOutput(List<T> accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<List<T>> getAccumulatorCoder(CoderRegistry registry, Coder<T> inputCoder) {
+      return ListCoder.of(inputCoder);
+    }
+
+    @Override
+    public Coder<List<T>> getDefaultOutputCoder(CoderRegistry registry, Coder<T> inputCoder) {
+      return ListCoder.of(inputCoder);
+    }
+  }
+
+  /**
+   * Creates a primitive {@link PCollectionView}.
+   *
+   * <p>For internal use only by runner implementors.
+   *
+   * @param <ElemT> The type of the elements of the input PCollection
+   * @param <ViewT> The type associated with the {@link PCollectionView} used as a side input
+   */
+  public static class CreateFlinkPCollectionView<ElemT, ViewT>
+      extends PTransform<PCollection<List<ElemT>>, PCollectionView<ViewT>> {
+    private PCollectionView<ViewT> view;
+
+    private CreateFlinkPCollectionView(PCollectionView<ViewT> view) {
+      this.view = view;
+    }
+
+    public static <ElemT, ViewT> CreateFlinkPCollectionView<ElemT, ViewT> of(
+        PCollectionView<ViewT> view) {
+      return new CreateFlinkPCollectionView<>(view);
+    }
+
+    @Override
+    public PCollectionView<ViewT> expand(PCollection<List<ElemT>> input) {
+      return view;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
new file mode 100644
index 0000000..3acc3ea
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline.
+ */
+class PipelineTranslationOptimizer extends FlinkPipelineTranslator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslationOptimizer.class);
+
+  private TranslationMode translationMode;
+
+  private final FlinkPipelineOptions options;
+
+  public PipelineTranslationOptimizer(TranslationMode defaultMode, FlinkPipelineOptions options) {
+    this.translationMode = defaultMode;
+    this.options = options;
+  }
+
+  public TranslationMode getTranslationMode() {
+
+    // override user-specified translation mode
+    if (options.isStreaming()) {
+      return TranslationMode.STREAMING;
+    }
+
+    return translationMode;
+  }
+
+  @Override
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
+    return CompositeBehavior.ENTER_TRANSFORM;
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {}
+
+  @Override
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+    Class<? extends PTransform> transformClass = node.getTransform().getClass();
+    if (transformClass == Read.Unbounded.class) {
+      LOG.info("Found {}. Switching to streaming execution.", transformClass);
+      translationMode = TranslationMode.STREAMING;
+    }
+  }
+
+  @Override
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {}
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
new file mode 100644
index 0000000..8f50105
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.PipelineResult;
+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.PipelineRunner;
+import org.apache.beam.sdk.util.UserCodeException;
+
+/**
+ * Test Flink runner.
+ */
+public class TestFlinkRunner extends PipelineRunner<PipelineResult> {
+
+  private FlinkRunner delegate;
+
+  private TestFlinkRunner(FlinkPipelineOptions options) {
+    // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment
+    options.setFlinkMaster("[auto]");
+    this.delegate = FlinkRunner.fromOptions(options);
+  }
+
+  public static TestFlinkRunner fromOptions(PipelineOptions options) {
+    FlinkPipelineOptions flinkOptions =
+        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+    return new TestFlinkRunner(flinkOptions);
+  }
+
+  public static TestFlinkRunner create(boolean streaming) {
+    FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+    flinkOptions.setRunner(TestFlinkRunner.class);
+    flinkOptions.setStreaming(streaming);
+    return TestFlinkRunner.fromOptions(flinkOptions);
+  }
+
+  @Override
+  public PipelineResult run(Pipeline pipeline) {
+    try {
+      return delegate.run(pipeline);
+    } catch (Throwable t) {
+      // Special case hack to pull out assertion errors from PAssert; instead there should
+      // probably be a better story along the lines of UserCodeException.
+      UserCodeException innermostUserCodeException = null;
+      Throwable current = t;
+      for (; current.getCause() != null; current = current.getCause()) {
+        if (current instanceof UserCodeException) {
+          innermostUserCodeException = ((UserCodeException) current);
+        }
+      }
+      if (innermostUserCodeException != null) {
+        current = innermostUserCodeException.getCause();
+      }
+      if (current instanceof AssertionError) {
+        throw (AssertionError) current;
+      }
+      throw new PipelineExecutionException(current);
+    }
+  }
+
+  public PipelineOptions getPipelineOptions() {
+    return delegate.getPipelineOptions();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
new file mode 100644
index 0000000..ad54750
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+/**
+ * The translation mode of the Beam Pipeline.
+ */
+enum TranslationMode {
+
+  /** Uses the batch mode of Flink. */
+  BATCH,
+
+  /** Uses the streaming mode of Flink. */
+  STREAMING
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/package-info.java
new file mode 100644
index 0000000..57f1e59
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.java
new file mode 100644
index 0000000..fb2493b
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAggregatorFactory.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 org.apache.beam.runners.flink.translation.functions;
+
+import org.apache.beam.runners.core.AggregatorFactory;
+import org.apache.beam.runners.core.ExecutionContext;
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/**
+ * A {@link AggregatorFactory} for the Flink Batch Runner.
+ */
+public class FlinkAggregatorFactory implements AggregatorFactory{
+
+  private final RuntimeContext runtimeContext;
+
+  public FlinkAggregatorFactory(RuntimeContext runtimeContext) {
+    this.runtimeContext = runtimeContext;
+  }
+
+  @Override
+  public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
+      Class<?> fnClass, ExecutionContext.StepContext stepContext, String aggregatorName,
+      Combine.CombineFn<InputT, AccumT, OutputT> combine) {
+    @SuppressWarnings("unchecked")
+    SerializableFnAggregatorWrapper<InputT, OutputT> result =
+        (SerializableFnAggregatorWrapper<InputT, OutputT>)
+            runtimeContext.getAccumulator(aggregatorName);
+
+    if (result == null) {
+      result = new SerializableFnAggregatorWrapper<>(combine);
+      runtimeContext.addAccumulator(aggregatorName, result);
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
new file mode 100644
index 0000000..447b1e5
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.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 org.apache.beam.runners.flink.translation.functions;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.collect.Iterables;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.joda.time.Instant;
+
+/**
+ * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for
+ * Flink functions.
+ */
+class FlinkAssignContext<InputT, W extends BoundedWindow>
+    extends WindowFn<InputT, W>.AssignContext {
+  private final WindowedValue<InputT> value;
+
+  FlinkAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
+    fn.super();
+    checkArgument(
+        Iterables.size(value.getWindows()) == 1,
+        String.format(
+            "%s passed to window assignment must be in a single window, but it was in %s: %s",
+            WindowedValue.class.getSimpleName(),
+            Iterables.size(value.getWindows()),
+            value.getWindows()));
+    this.value = value;
+  }
+
+  @Override
+  public InputT element() {
+    return value.getValue();
+  }
+
+  @Override
+  public Instant timestamp() {
+    return value.getTimestamp();
+  }
+
+  @Override
+  public BoundedWindow window() {
+    return Iterables.getOnlyElement(value.getWindows());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java
new file mode 100644
index 0000000..c3a5095
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.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 org.apache.beam.runners.flink.translation.functions;
+
+import java.util.Collection;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Flink {@link FlatMapFunction} for implementing
+ * {@link org.apache.beam.sdk.transforms.windowing.Window.Assign}.
+ */
+public class FlinkAssignWindows<T, W extends BoundedWindow>
+    implements FlatMapFunction<WindowedValue<T>, WindowedValue<T>> {
+
+  private final WindowFn<T, W> windowFn;
+
+  public FlinkAssignWindows(WindowFn<T, W> windowFn) {
+    this.windowFn = windowFn;
+  }
+
+  @Override
+  public void flatMap(
+      WindowedValue<T> input, Collector<WindowedValue<T>> collector) throws Exception {
+    Collection<W> windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input));
+    for (W window: windows) {
+      collector.collect(
+          WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
new file mode 100644
index 0000000..51582af
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.functions;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+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.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+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.TupleTag;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+
+/**
+ * Encapsulates a {@link DoFn}
+ * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
+ *
+ * <p>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.
+ */
+public class FlinkDoFnFunction<InputT, OutputT>
+    extends RichMapPartitionFunction<WindowedValue<InputT>, WindowedValue<OutputT>> {
+
+  private final SerializedPipelineOptions serializedOptions;
+
+  private final DoFn<InputT, OutputT> doFn;
+  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
+
+  private final WindowingStrategy<?, ?> windowingStrategy;
+
+  private final Map<TupleTag<?>, Integer> outputMap;
+  private final TupleTag<OutputT> mainOutputTag;
+
+  private transient DoFnInvoker<InputT, OutputT> doFnInvoker;
+
+  public FlinkDoFnFunction(
+      DoFn<InputT, OutputT> doFn,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
+      PipelineOptions options,
+      Map<TupleTag<?>, Integer> outputMap,
+      TupleTag<OutputT> mainOutputTag) {
+
+    this.doFn = doFn;
+    this.sideInputs = sideInputs;
+    this.serializedOptions = new SerializedPipelineOptions(options);
+    this.windowingStrategy = windowingStrategy;
+    this.outputMap = outputMap;
+    this.mainOutputTag = mainOutputTag;
+
+  }
+
+  @Override
+  public void mapPartition(
+      Iterable<WindowedValue<InputT>> values,
+      Collector<WindowedValue<OutputT>> out) throws Exception {
+
+    RuntimeContext runtimeContext = getRuntimeContext();
+
+    DoFnRunners.OutputManager outputManager;
+    if (outputMap == null) {
+      outputManager = new FlinkDoFnFunction.DoFnOutputManager(out);
+    } else {
+      // it has some additional outputs
+      outputManager =
+          new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap);
+    }
+
+    DoFnRunner<InputT, OutputT> doFnRunner = DoFnRunners.simpleRunner(
+        serializedOptions.getPipelineOptions(), doFn,
+        new FlinkSideInputReader(sideInputs, runtimeContext),
+        outputManager,
+        mainOutputTag,
+        // see SimpleDoFnRunner, just use it to limit number of additional outputs
+        Collections.<TupleTag<?>>emptyList(),
+        new FlinkNoOpStepContext(),
+        new FlinkAggregatorFactory(runtimeContext),
+        windowingStrategy);
+
+    doFnRunner.startBundle();
+
+    for (WindowedValue<InputT> value : values) {
+      doFnRunner.processElement(value);
+    }
+
+    doFnRunner.finishBundle();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    doFnInvoker = DoFnInvokers.invokerFor(doFn);
+    doFnInvoker.invokeSetup();
+  }
+
+  @Override
+  public void close() throws Exception {
+    doFnInvoker.invokeTeardown();
+  }
+
+  static class DoFnOutputManager
+      implements DoFnRunners.OutputManager {
+
+    private Collector collector;
+
+    DoFnOutputManager(Collector collector) {
+      this.collector = collector;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      collector.collect(output);
+    }
+  }
+
+  static class MultiDoFnOutputManager
+      implements DoFnRunners.OutputManager {
+
+    private Collector<WindowedValue<RawUnionValue>> collector;
+    private Map<TupleTag<?>, Integer> outputMap;
+
+    MultiDoFnOutputManager(Collector<WindowedValue<RawUnionValue>> collector,
+                      Map<TupleTag<?>, Integer> outputMap) {
+      this.collector = collector;
+      this.outputMap = outputMap;
+    }
+
+    @Override
+    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      collector.collect(WindowedValue.of(new RawUnionValue(outputMap.get(tag), output.getValue()),
+          output.getTimestamp(), output.getWindows(), output.getPane()));
+    }
+  }
+
+}


[13/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
deleted file mode 100644
index af4b354..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 9b449aa..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.coders.Coder;
-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;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
- * Dataflow {@link org.apache.beam.sdk.coders.Coder}s.
- */
-public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
-
-  private final Coder<T> coder;
-
-  public CoderTypeInformation(Coder<T> coder) {
-    checkNotNull(coder);
-    this.coder = coder;
-  }
-
-  public Coder<T> getCoder() {
-    return coder;
-  }
-
-  @Override
-  public boolean isBasicType() {
-    return false;
-  }
-
-  @Override
-  public boolean isTupleType() {
-    return false;
-  }
-
-  @Override
-  public int getArity() {
-    return 1;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public Class<T> getTypeClass() {
-    // We don't have the Class, so we have to pass null here. What a shame...
-    return (Class<T>) Object.class;
-  }
-
-  @Override
-  public boolean isKeyType() {
-    return true;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public TypeSerializer<T> createSerializer(ExecutionConfig config) {
-    return new CoderTypeSerializer<>(coder);
-  }
-
-  @Override
-  public int getTotalFields() {
-    return 2;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    CoderTypeInformation that = (CoderTypeInformation) o;
-
-    return coder.equals(that.coder);
-
-  }
-
-  @Override
-  public int hashCode() {
-    return coder.hashCode();
-  }
-
-  @Override
-  public boolean canEqual(Object obj) {
-    return obj instanceof CoderTypeInformation;
-  }
-
-  @Override
-  public String toString() {
-    return "CoderTypeInformation{coder=" + coder + '}';
-  }
-
-  @Override
-  public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
-      executionConfig) {
-    throw new UnsupportedOperationException(
-        "Non-encoded values cannot be compared directly.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index e210ed9..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import java.io.EOFException;
-import java.io.IOException;
-import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
-import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
- * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}.
- */
-public class CoderTypeSerializer<T> extends TypeSerializer<T> {
-
-  private Coder<T> coder;
-
-  public CoderTypeSerializer(Coder<T> coder) {
-    this.coder = coder;
-  }
-
-  @Override
-  public boolean isImmutableType() {
-    return false;
-  }
-
-  @Override
-  public CoderTypeSerializer<T> duplicate() {
-    return new CoderTypeSerializer<>(coder);
-  }
-
-  @Override
-  public T createInstance() {
-    return null;
-  }
-
-  @Override
-  public T copy(T t) {
-    try {
-      return CoderUtils.clone(coder, t);
-    } catch (CoderException e) {
-      throw new RuntimeException("Could not clone.", e);
-    }
-  }
-
-  @Override
-  public T copy(T t, T reuse) {
-    return copy(t);
-  }
-
-  @Override
-  public int getLength() {
-    return -1;
-  }
-
-  @Override
-  public void serialize(T t, DataOutputView dataOutputView) throws IOException {
-    DataOutputViewWrapper outputWrapper = new DataOutputViewWrapper(dataOutputView);
-    coder.encode(t, outputWrapper, Coder.Context.NESTED);
-  }
-
-  @Override
-  public T deserialize(DataInputView dataInputView) throws IOException {
-    try {
-      DataInputViewWrapper inputWrapper = new DataInputViewWrapper(dataInputView);
-      return coder.decode(inputWrapper, Coder.Context.NESTED);
-    } catch (CoderException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof EOFException) {
-        throw (EOFException) cause;
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  @Override
-  public T deserialize(T t, DataInputView dataInputView) throws IOException {
-    return deserialize(dataInputView);
-  }
-
-  @Override
-  public void copy(
-      DataInputView dataInputView,
-      DataOutputView dataOutputView) throws IOException {
-    serialize(deserialize(dataInputView), dataOutputView);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    CoderTypeSerializer that = (CoderTypeSerializer) o;
-    return coder.equals(that.coder);
-  }
-
-  @Override
-  public boolean canEqual(Object obj) {
-    return obj instanceof CoderTypeSerializer;
-  }
-
-  @Override
-  public int hashCode() {
-    return coder.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
deleted file mode 100644
index 667ef45..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import java.io.IOException;
-import java.util.Arrays;
-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;
-import org.apache.flink.core.memory.MemorySegment;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have
- * been encoded to byte data by a {@link Coder}.
- */
-public class EncodedValueComparator extends TypeComparator<byte[]> {
-
-  /** For storing the Reference in encoded form. */
-  private transient byte[] encodedReferenceKey;
-
-  private final boolean ascending;
-
-  public EncodedValueComparator(boolean ascending) {
-    this.ascending = ascending;
-  }
-
-  @Override
-  public int hash(byte[] record) {
-    return Arrays.hashCode(record);
-  }
-
-  @Override
-  public void setReference(byte[] toCompare) {
-    this.encodedReferenceKey = toCompare;
-  }
-
-  @Override
-  public boolean equalToReference(byte[] candidate) {
-    if (encodedReferenceKey.length != candidate.length) {
-      return false;
-    }
-    int len = candidate.length;
-    for (int i = 0; i < len; i++) {
-      if (encodedReferenceKey[i] != candidate[i]) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public int compareToReference(TypeComparator<byte[]> other) {
-    // VERY IMPORTANT: compareToReference does not behave like Comparable.compare
-    // the meaning of the return value is inverted.
-
-    EncodedValueComparator otherEncodedValueComparator = (EncodedValueComparator) other;
-
-    int len = Math.min(
-        encodedReferenceKey.length,
-        otherEncodedValueComparator.encodedReferenceKey.length);
-
-    for (int i = 0; i < len; i++) {
-      byte b1 = encodedReferenceKey[i];
-      byte b2 = otherEncodedValueComparator.encodedReferenceKey[i];
-      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
-      if (result != 0) {
-        return ascending ? -result : result;
-      }
-    }
-    int result =
-        encodedReferenceKey.length - otherEncodedValueComparator.encodedReferenceKey.length;
-    return ascending ? -result : result;
-  }
-
-
-  @Override
-  public int compare(byte[] first, byte[] second) {
-    int len = Math.min(first.length, second.length);
-    for (int i = 0; i < len; i++) {
-      byte b1 = first[i];
-      byte b2 = second[i];
-      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
-      if (result != 0) {
-        return ascending ? result : -result;
-      }
-    }
-    int result = first.length - second.length;
-    return ascending ? result : -result;
-  }
-
-  @Override
-  public int compareSerialized(
-      DataInputView firstSource,
-      DataInputView secondSource) throws IOException {
-    int lengthFirst = firstSource.readInt();
-    int lengthSecond = secondSource.readInt();
-
-    int len = Math.min(lengthFirst, lengthSecond);
-    for (int i = 0; i < len; i++) {
-      byte b1 = firstSource.readByte();
-      byte b2 = secondSource.readByte();
-      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
-      if (result != 0) {
-        return ascending ? result : -result;
-      }
-    }
-
-    int result = lengthFirst - lengthSecond;
-    return ascending ? result : -result;
-  }
-
-
-
-  @Override
-  public boolean supportsNormalizedKey() {
-    // disabled because this seems to not work with some coders,
-    // such as the AvroCoder
-    return false;
-  }
-
-  @Override
-  public boolean supportsSerializationWithKeyNormalization() {
-    return false;
-  }
-
-  @Override
-  public int getNormalizeKeyLen() {
-    return Integer.MAX_VALUE;
-  }
-
-  @Override
-  public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-    return true;
-  }
-
-  @Override
-  public void putNormalizedKey(byte[] record, MemorySegment target, int offset, int numBytes) {
-    final int limit = offset + numBytes;
-
-    target.put(offset, record, 0, Math.min(numBytes, record.length));
-
-    offset += record.length;
-
-    while (offset < limit) {
-      target.put(offset++, (byte) 0);
-    }
-  }
-
-  @Override
-  public void writeWithKeyNormalization(byte[] record, DataOutputView target) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public byte[] readWithKeyDenormalization(byte[] reuse, DataInputView source) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean invertNormalizedKey() {
-    return !ascending;
-  }
-
-  @Override
-  public TypeComparator<byte[]> duplicate() {
-    return new EncodedValueComparator(ascending);
-  }
-
-  @Override
-  public int extractKeys(Object record, Object[] target, int index) {
-    target[index] = record;
-    return 1;
-  }
-
-  @Override
-  public TypeComparator[] getFlatComparators() {
-    return new TypeComparator[] { this.duplicate() };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
deleted file mode 100644
index 41db61e..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import java.io.IOException;
-
-import org.apache.beam.sdk.coders.Coder;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-/**
- * {@link TypeSerializer} for values that were encoded using a {@link Coder}.
- */
-public final class EncodedValueSerializer extends TypeSerializer<byte[]> {
-
-  private static final long serialVersionUID = 1L;
-
-  private static final byte[] EMPTY = new byte[0];
-
-  @Override
-  public boolean isImmutableType() {
-    return true;
-  }
-
-  @Override
-  public byte[] createInstance() {
-    return EMPTY;
-  }
-
-  @Override
-  public byte[] copy(byte[] from) {
-    return from;
-  }
-
-  @Override
-  public byte[] copy(byte[] from, byte[] reuse) {
-    return copy(from);
-  }
-
-  @Override
-  public int getLength() {
-    return -1;
-  }
-
-
-  @Override
-  public void serialize(byte[] record, DataOutputView target) throws IOException {
-    if (record == null) {
-      throw new IllegalArgumentException("The record must not be null.");
-    }
-
-    final int len = record.length;
-    target.writeInt(len);
-    target.write(record);
-  }
-
-  @Override
-  public byte[] deserialize(DataInputView source) throws IOException {
-    final int len = source.readInt();
-    byte[] result = new byte[len];
-    source.readFully(result);
-    return result;
-  }
-
-  @Override
-  public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException {
-    return deserialize(source);
-  }
-
-  @Override
-  public void copy(DataInputView source, DataOutputView target) throws IOException {
-    final int len = source.readInt();
-    target.writeInt(len);
-    target.write(source, len);
-  }
-
-  @Override
-  public boolean canEqual(Object obj) {
-    return obj instanceof EncodedValueSerializer;
-  }
-
-  @Override
-  public int hashCode() {
-    return this.getClass().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return obj instanceof EncodedValueSerializer;
-  }
-
-  @Override
-  public TypeSerializer<byte[]> duplicate() {
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
deleted file mode 100644
index e24bf31..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.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 org.apache.beam.runners.flink.translation.types;
-
-import org.apache.beam.sdk.coders.Coder;
-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;
-
-/**
- * Flink {@link TypeInformation} for Beam values that have been encoded to byte data
- * by a {@link Coder}.
- */
-public class EncodedValueTypeInformation
-    extends TypeInformation<byte[]>
-    implements AtomicType<byte[]> {
-
-  private static final long serialVersionUID = 1L;
-
-  @Override
-  public boolean isBasicType() {
-    return false;
-  }
-
-  @Override
-  public boolean isTupleType() {
-    return false;
-  }
-
-  @Override
-  public int getArity() {
-    return 0;
-  }
-
-  @Override
-  public int getTotalFields() {
-    return 0;
-  }
-
-  @Override
-  public Class<byte[]> getTypeClass() {
-    return byte[].class;
-  }
-
-  @Override
-  public boolean isKeyType() {
-    return true;
-  }
-
-  @Override
-  public TypeSerializer<byte[]> createSerializer(ExecutionConfig executionConfig) {
-    return new EncodedValueSerializer();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return other instanceof EncodedValueTypeInformation;
-  }
-
-  @Override
-  public int hashCode() {
-    return this.getClass().hashCode();
-  }
-
-  @Override
-  public boolean canEqual(Object obj) {
-    return obj instanceof EncodedValueTypeInformation;
-  }
-
-  @Override
-  public String toString() {
-    return "EncodedValueTypeInformation";
-  }
-
-  @Override
-  public TypeComparator<byte[]> createComparator(
-      boolean sortOrderAscending,
-      ExecutionConfig executionConfig) {
-    return new EncodedValueComparator(sortOrderAscending);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
deleted file mode 100644
index 36b5ba3..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.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 org.apache.beam.runners.flink.translation.types;
-
-import java.io.ByteArrayOutputStream;
-
-/**
- * Version of {@link java.io.ByteArrayOutputStream} that allows to retrieve the internal
- * byte[] buffer without incurring an array copy.
- */
-public class InspectableByteArrayOutputStream extends ByteArrayOutputStream {
-
-  /**
-   * Get the underlying byte array.
-   */
-  public byte[] getBuffer() {
-    return buf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
deleted file mode 100644
index 9df6836..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.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 org.apache.beam.runners.flink.translation.types;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.util.CoderUtils;
-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;
-
-/**
- * {@link KeySelector} that extracts the key from a {@link KV} and returns
- * it in encoded form as a {@code byte} array.
- */
-public class KvKeySelector<InputT, K>
-    implements KeySelector<WindowedValue<KV<K, InputT>>, byte[]>, ResultTypeQueryable<byte[]> {
-
-  private final Coder<K> keyCoder;
-
-  public KvKeySelector(Coder<K> keyCoder) {
-    this.keyCoder = keyCoder;
-  }
-
-  @Override
-  public byte[] getKey(WindowedValue<KV<K, InputT>> value) throws Exception {
-    return CoderUtils.encodeToByteArray(keyCoder, value.getValue().getKey());
-  }
-
-  @Override
-  public TypeInformation<byte[]> getProducedType() {
-    return new EncodedValueTypeInformation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
deleted file mode 100644
index 6fb3182..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.types;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
deleted file mode 100644
index 2256bb1..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.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 org.apache.beam.runners.flink.translation.utils;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import org.apache.beam.sdk.io.FileSystems;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.IOChannelUtils;
-
-/**
- * Encapsulates the PipelineOptions in serialized form to ship them to the cluster.
- */
-public class SerializedPipelineOptions implements Serializable {
-
-  private final byte[] serializedOptions;
-
-  /** Lazily initialized copy of deserialized options. */
-  private transient PipelineOptions pipelineOptions;
-
-  public SerializedPipelineOptions(PipelineOptions options) {
-    checkNotNull(options, "PipelineOptions must not be null.");
-
-    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
-      new ObjectMapper().writeValue(baos, options);
-      this.serializedOptions = baos.toByteArray();
-    } catch (Exception e) {
-      throw new RuntimeException("Couldn't serialize PipelineOptions.", e);
-    }
-
-  }
-
-  public PipelineOptions getPipelineOptions() {
-    if (pipelineOptions == null) {
-      try {
-        pipelineOptions = new ObjectMapper().readValue(serializedOptions, PipelineOptions.class);
-
-        IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions);
-        FileSystems.setDefaultConfigInWorkers(pipelineOptions);
-      } catch (IOException e) {
-        throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e);
-      }
-    }
-
-    return pipelineOptions;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
deleted file mode 100644
index 5dedd53..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 82a2c4e..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.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 org.apache.beam.runners.flink.translation.wrappers;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import org.apache.flink.core.memory.DataInputView;
-
-/**
- * Wrapper for {@link DataInputView}. We need this because Flink reads data using a
- * {@link org.apache.flink.core.memory.DataInputView} while
- * Dataflow {@link org.apache.beam.sdk.coders.Coder}s expect an
- * {@link java.io.InputStream}.
- */
-public class DataInputViewWrapper extends InputStream {
-
-  private DataInputView inputView;
-
-  public DataInputViewWrapper(DataInputView inputView) {
-    this.inputView = inputView;
-  }
-
-  public void setInputView(DataInputView inputView) {
-    this.inputView = inputView;
-  }
-
-  @Override
-  public int read() throws IOException {
-    try {
-      return inputView.readUnsignedByte();
-    } catch (EOFException e) {
-      // translate between DataInput and InputStream,
-      // DataInput signals EOF by exception, InputStream does it by returning -1
-      return -1;
-    }
-  }
-
-  @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    return inputView.read(b, off, len);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index f2d9db2..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.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 org.apache.beam.runners.flink.translation.wrappers;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import org.apache.flink.core.memory.DataOutputView;
-
-/**
- * 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 org.apache.beam.sdk.coders.Coder}s expect an
- * {@link java.io.OutputStream}.
- */
-public class DataOutputViewWrapper extends OutputStream {
-
-  private DataOutputView outputView;
-
-  public DataOutputViewWrapper(DataOutputView outputView) {
-    this.outputView = outputView;
-  }
-
-  public void setOutputView(DataOutputView outputView) {
-    this.outputView = outputView;
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    outputView.write(b);
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    outputView.write(b, off, len);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 70d97e3..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.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 org.apache.beam.runners.flink.translation.wrappers;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import java.io.Serializable;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.flink.api.common.accumulators.Accumulator;
-
-/**
- * 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 org.apache.beam.sdk.transforms.ParDo}
- * operation.
- */
-public class SerializableFnAggregatorWrapper<InputT, OutputT>
-    implements Aggregator<InputT, OutputT>, Accumulator<InputT, Serializable> {
-
-  private OutputT aa;
-  private Combine.CombineFn<InputT, ?, OutputT> combiner;
-
-  public SerializableFnAggregatorWrapper(Combine.CombineFn<InputT, ?, OutputT> combiner) {
-    this.combiner = combiner;
-    resetLocal();
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void add(InputT value) {
-    this.aa = combiner.apply(ImmutableList.of((InputT) aa, value));
-  }
-
-  @Override
-  public Serializable getLocalValue() {
-    return (Serializable) aa;
-  }
-
-  @Override
-  public void resetLocal() {
-    this.aa = combiner.apply(ImmutableList.<InputT>of());
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void merge(Accumulator<InputT, Serializable> other) {
-    this.aa = combiner.apply(ImmutableList.of((InputT) aa, (InputT) other.getLocalValue()));
-  }
-
-  @Override
-  public void addValue(InputT value) {
-    add(value);
-  }
-
-  @Override
-  public String getName() {
-    return "Aggregator :" + combiner.toString();
-  }
-
-  @Override
-  public Combine.CombineFn<InputT, ?, OutputT> getCombineFn() {
-    return combiner;
-  }
-
-  @Override
-  public Accumulator<InputT, Serializable> clone() {
-    try {
-      super.clone();
-    } catch (CloneNotSupportedException e) {
-      // Flink Accumulators cannot throw CloneNotSupportedException, work around that.
-      throw new RuntimeException(e);
-    }
-
-    // copy it by merging
-    OutputT resultCopy = combiner.apply(Lists.newArrayList((InputT) aa));
-    SerializableFnAggregatorWrapper<InputT, OutputT> result = new
-        SerializableFnAggregatorWrapper<>(combiner);
-
-    result.aa = resultCopy;
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index a87472b..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers;
-
-import java.io.IOException;
-import java.util.List;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-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.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}.
- */
-public class SourceInputFormat<T>
-    implements InputFormat<WindowedValue<T>, SourceInputSplit<T>> {
-  private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
-
-  private final BoundedSource<T> initialSource;
-
-  private transient PipelineOptions options;
-  private final SerializedPipelineOptions serializedOptions;
-
-  private transient BoundedSource.BoundedReader<T> reader;
-  private boolean inputAvailable = false;
-
-  public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
-    this.initialSource = initialSource;
-    this.serializedOptions = new SerializedPipelineOptions(options);
-  }
-
-  @Override
-  public void configure(Configuration configuration) {
-    options = serializedOptions.getPipelineOptions();
-  }
-
-  @Override
-  public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
-    reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
-    inputAvailable = reader.start();
-  }
-
-  @Override
-  public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
-    try {
-      final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
-
-      return new BaseStatistics() {
-        @Override
-        public long getTotalInputSize() {
-          return estimatedSize;
-        }
-
-        @Override
-        public long getNumberOfRecords() {
-          return BaseStatistics.NUM_RECORDS_UNKNOWN;
-        }
-
-        @Override
-        public float getAverageRecordWidth() {
-          return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
-        }
-      };
-    } catch (Exception e) {
-      LOG.warn("Could not read Source statistics: {}", e);
-    }
-
-    return null;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
-    try {
-      long desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
-      List<? extends Source<T>> shards =
-          initialSource.split(desiredSizeBytes, options);
-      int numShards = shards.size();
-      SourceInputSplit<T>[] sourceInputSplits = new SourceInputSplit[numShards];
-      for (int i = 0; i < numShards; i++) {
-        sourceInputSplits[i] = new SourceInputSplit<>(shards.get(i), i);
-      }
-      return sourceInputSplits;
-    } catch (Exception e) {
-      throw new IOException("Could not create input splits from Source.", e);
-    }
-  }
-
-  @Override
-  public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
-    return new DefaultInputSplitAssigner(sourceInputSplits);
-  }
-
-
-  @Override
-  public boolean reachedEnd() throws IOException {
-    return !inputAvailable;
-  }
-
-  @Override
-  public WindowedValue<T> nextRecord(WindowedValue<T> t) throws IOException {
-    if (inputAvailable) {
-      final T current = reader.getCurrent();
-      final Instant timestamp = reader.getCurrentTimestamp();
-      // advance reader to have a record ready next time
-      inputAvailable = reader.advance();
-      return WindowedValue.of(
-          current,
-          timestamp,
-          GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
-    }
-
-    return null;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // TODO null check can be removed once FLINK-3796 is fixed
-    if (reader != null) {
-      reader.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index e4a7386..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.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 org.apache.beam.runners.flink.translation.wrappers;
-
-import org.apache.beam.sdk.io.Source;
-import org.apache.flink.core.io.InputSplit;
-
-/**
- * {@link org.apache.flink.core.io.InputSplit} for
- * {@link org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat}. We pass
- * the sharded Source around in the input split because Sources simply split up into several
- * Sources for sharding. This is different to how Flink creates a separate InputSplit from
- * an InputFormat.
- */
-public class SourceInputSplit<T> implements InputSplit {
-
-  private Source<T> source;
-  private int splitNumber;
-
-  public SourceInputSplit() {
-  }
-
-  public SourceInputSplit(Source<T> source, int splitNumber) {
-    this.source = source;
-    this.splitNumber = splitNumber;
-  }
-
-  @Override
-  public int getSplitNumber() {
-    return splitNumber;
-  }
-
-  public Source<T> getSource() {
-    return source;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
deleted file mode 100644
index 72f7deb..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.wrappers;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
deleted file mode 100644
index 8a09286..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ /dev/null
@@ -1,774 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterables;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.AggregatorFactory;
-import org.apache.beam.runners.core.DoFnRunner;
-import org.apache.beam.runners.core.DoFnRunners;
-import org.apache.beam.runners.core.ExecutionContext;
-import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
-import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
-import org.apache.beam.runners.core.SideInputHandler;
-import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.StateNamespaces.WindowNamespace;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.core.StatefulDoFnRunner;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.KeyGroupCheckpointedOperator;
-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.join.RawUnionValue;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.NullSideInputReader;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
-import org.apache.flink.runtime.state.KeyGroupsList;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
-import org.apache.flink.runtime.state.StateInitializationContext;
-import org.apache.flink.runtime.state.StateSnapshotContext;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.HeapInternalTimerService;
-import org.apache.flink.streaming.api.operators.InternalTimer;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.Triggerable;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.joda.time.Instant;
-
-/**
- * Flink operator for executing {@link DoFn DoFns}.
- *
- * @param <InputT> the input type of the {@link DoFn}
- * @param <FnOutputT> the output type of the {@link DoFn}
- * @param <OutputT> the output type of the operator, this can be different from the fn output
- *                 type when we have additional tagged outputs
- */
-public class DoFnOperator<InputT, FnOutputT, OutputT>
-    extends AbstractStreamOperator<OutputT>
-    implements OneInputStreamOperator<WindowedValue<InputT>, OutputT>,
-      TwoInputStreamOperator<WindowedValue<InputT>, RawUnionValue, OutputT>,
-    KeyGroupCheckpointedOperator, Triggerable<Object, TimerData> {
-
-  protected DoFn<InputT, FnOutputT> doFn;
-
-  protected final SerializedPipelineOptions serializedOptions;
-
-  protected final TupleTag<FnOutputT> mainOutputTag;
-  protected final List<TupleTag<?>> additionalOutputTags;
-
-  protected final Collection<PCollectionView<?>> sideInputs;
-  protected final Map<Integer, PCollectionView<?>> sideInputTagMapping;
-
-  protected final WindowingStrategy<?, ?> windowingStrategy;
-
-  protected final OutputManagerFactory<OutputT> outputManagerFactory;
-
-  protected transient DoFnRunner<InputT, FnOutputT> doFnRunner;
-  protected transient PushbackSideInputDoFnRunner<InputT, FnOutputT> pushbackDoFnRunner;
-
-  protected transient SideInputHandler sideInputHandler;
-
-  protected transient SideInputReader sideInputReader;
-
-  protected transient DoFnRunners.OutputManager outputManager;
-
-  private transient DoFnInvoker<InputT, FnOutputT> doFnInvoker;
-
-  protected transient long currentInputWatermark;
-
-  protected transient long currentOutputWatermark;
-
-  private transient StateTag<Object, BagState<WindowedValue<InputT>>> pushedBackTag;
-
-  protected transient FlinkStateInternals<?> stateInternals;
-
-  private Coder<WindowedValue<InputT>> inputCoder;
-
-  private final Coder<?> keyCoder;
-
-  private final TimerInternals.TimerDataCoder timerCoder;
-
-  protected transient HeapInternalTimerService<?, TimerInternals.TimerData> timerService;
-
-  protected transient FlinkTimerInternals timerInternals;
-
-  private transient StateInternals<?> pushbackStateInternals;
-
-  private transient Optional<Long> pushedBackWatermark;
-
-  public DoFnOperator(
-      DoFn<InputT, FnOutputT> doFn,
-      Coder<WindowedValue<InputT>> inputCoder,
-      TupleTag<FnOutputT> mainOutputTag,
-      List<TupleTag<?>> additionalOutputTags,
-      OutputManagerFactory<OutputT> outputManagerFactory,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<Integer, PCollectionView<?>> sideInputTagMapping,
-      Collection<PCollectionView<?>> sideInputs,
-      PipelineOptions options,
-      Coder<?> keyCoder) {
-    this.doFn = doFn;
-    this.inputCoder = inputCoder;
-    this.mainOutputTag = mainOutputTag;
-    this.additionalOutputTags = additionalOutputTags;
-    this.sideInputTagMapping = sideInputTagMapping;
-    this.sideInputs = sideInputs;
-    this.serializedOptions = new SerializedPipelineOptions(options);
-    this.windowingStrategy = windowingStrategy;
-    this.outputManagerFactory = outputManagerFactory;
-
-    setChainingStrategy(ChainingStrategy.ALWAYS);
-
-    this.keyCoder = keyCoder;
-
-    this.timerCoder =
-        TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
-  }
-
-  private ExecutionContext.StepContext createStepContext() {
-    return new StepContext();
-  }
-
-  // allow overriding this in WindowDoFnOperator because this one dynamically creates
-  // the DoFn
-  protected DoFn<InputT, FnOutputT> getDoFn() {
-    return doFn;
-  }
-
-  @Override
-  public void open() throws Exception {
-    super.open();
-
-    currentInputWatermark = Long.MIN_VALUE;
-    currentOutputWatermark = Long.MIN_VALUE;
-
-    AggregatorFactory aggregatorFactory = new AggregatorFactory() {
-      @Override
-      public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
-          Class<?> fnClass,
-          ExecutionContext.StepContext stepContext,
-          String aggregatorName,
-          Combine.CombineFn<InputT, AccumT, OutputT> combine) {
-
-        @SuppressWarnings("unchecked")
-        SerializableFnAggregatorWrapper<InputT, OutputT> result =
-            (SerializableFnAggregatorWrapper<InputT, OutputT>)
-                getRuntimeContext().getAccumulator(aggregatorName);
-
-        if (result == null) {
-          result = new SerializableFnAggregatorWrapper<>(combine);
-          getRuntimeContext().addAccumulator(aggregatorName, result);
-        }
-        return result;
-      }
-    };
-
-    sideInputReader = NullSideInputReader.of(sideInputs);
-
-    if (!sideInputs.isEmpty()) {
-
-      pushedBackTag = StateTags.bag("pushed-back-values", inputCoder);
-
-      FlinkBroadcastStateInternals sideInputStateInternals =
-          new FlinkBroadcastStateInternals<>(
-              getContainingTask().getIndexInSubtaskGroup(), getOperatorStateBackend());
-
-      sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals);
-      sideInputReader = sideInputHandler;
-
-      // maybe init by initializeState
-      if (pushbackStateInternals == null) {
-        if (keyCoder != null) {
-          pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder,
-              getKeyedStateBackend());
-        } else {
-          pushbackStateInternals =
-              new FlinkSplitStateInternals<Object>(getOperatorStateBackend());
-        }
-      }
-
-      pushedBackWatermark = Optional.absent();
-
-    }
-
-    outputManager = outputManagerFactory.create(output);
-
-    // StatefulPardo or WindowDoFn
-    if (keyCoder != null) {
-      stateInternals = new FlinkStateInternals<>((KeyedStateBackend) getKeyedStateBackend(),
-          keyCoder);
-
-      timerService = (HeapInternalTimerService<?, TimerInternals.TimerData>)
-          getInternalTimerService("beam-timer", new CoderTypeSerializer<>(timerCoder), this);
-
-      timerInternals = new FlinkTimerInternals();
-
-    }
-
-    // WindowDoFnOperator need use state and timer to get DoFn.
-    // So must wait StateInternals and TimerInternals ready.
-    this.doFn = getDoFn();
-    doFnInvoker = DoFnInvokers.invokerFor(doFn);
-
-    doFnInvoker.invokeSetup();
-
-    ExecutionContext.StepContext stepContext = createStepContext();
-
-    doFnRunner = DoFnRunners.simpleRunner(
-        serializedOptions.getPipelineOptions(),
-        doFn,
-        sideInputReader,
-        outputManager,
-        mainOutputTag,
-        additionalOutputTags,
-        stepContext,
-        aggregatorFactory,
-        windowingStrategy);
-
-    if (doFn instanceof GroupAlsoByWindowViaWindowSetNewDoFn) {
-      // When the doFn is this, we know it came from WindowDoFnOperator and
-      //   InputT = KeyedWorkItem<K, V>
-      //   OutputT = KV<K, V>
-      //
-      // for some K, V
-
-
-      doFnRunner = DoFnRunners.lateDataDroppingRunner(
-          (DoFnRunner) doFnRunner,
-          stepContext,
-          windowingStrategy,
-          ((GroupAlsoByWindowViaWindowSetNewDoFn) doFn).getDroppedDueToLatenessAggregator());
-    } else if (keyCoder != null) {
-      // It is a stateful DoFn
-
-      StatefulDoFnRunner.CleanupTimer cleanupTimer =
-          new StatefulDoFnRunner.TimeInternalsCleanupTimer(
-              stepContext.timerInternals(), windowingStrategy);
-
-      // we don't know the window type
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      Coder windowCoder = windowingStrategy.getWindowFn().windowCoder();
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      StatefulDoFnRunner.StateCleaner<?> stateCleaner =
-          new StatefulDoFnRunner.StateInternalsStateCleaner<>(
-              doFn, stepContext.stateInternals(), windowCoder);
-
-      doFnRunner = DoFnRunners.defaultStatefulDoFnRunner(
-          doFn,
-          doFnRunner,
-          stepContext,
-          aggregatorFactory,
-          windowingStrategy,
-          cleanupTimer,
-          stateCleaner);
-    }
-
-    pushbackDoFnRunner =
-        SimplePushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler);
-  }
-
-  @Override
-  public void close() throws Exception {
-    super.close();
-    doFnInvoker.invokeTeardown();
-  }
-
-  protected final long getPushbackWatermarkHold() {
-    // if we don't have side inputs we never hold the watermark
-    if (sideInputs.isEmpty()) {
-      return BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-    }
-
-    try {
-      checkInitPushedBackWatermark();
-      return pushedBackWatermark.get();
-    } catch (Exception e) {
-      throw new RuntimeException("Error retrieving pushed back watermark state.", e);
-    }
-  }
-
-  private void checkInitPushedBackWatermark() {
-    // init and restore from pushedBack state.
-    // Not done in initializeState, because OperatorState is not ready.
-    if (!pushedBackWatermark.isPresent()) {
-
-      BagState<WindowedValue<InputT>> pushedBack =
-          pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
-
-      long min = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-      for (WindowedValue<InputT> value : pushedBack.read()) {
-        min = Math.min(min, value.getTimestamp().getMillis());
-      }
-      setPushedBackWatermark(min);
-    }
-  }
-
-  @Override
-  public final void processElement(
-      StreamRecord<WindowedValue<InputT>> streamRecord) throws Exception {
-    doFnRunner.startBundle();
-    doFnRunner.processElement(streamRecord.getValue());
-    doFnRunner.finishBundle();
-  }
-
-  private void setPushedBackWatermark(long watermark) {
-    pushedBackWatermark = Optional.fromNullable(watermark);
-  }
-
-  @Override
-  public final void processElement1(
-      StreamRecord<WindowedValue<InputT>> streamRecord) throws Exception {
-    pushbackDoFnRunner.startBundle();
-    Iterable<WindowedValue<InputT>> justPushedBack =
-        pushbackDoFnRunner.processElementInReadyWindows(streamRecord.getValue());
-
-    BagState<WindowedValue<InputT>> pushedBack =
-        pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
-
-    checkInitPushedBackWatermark();
-
-    long min = pushedBackWatermark.get();
-    for (WindowedValue<InputT> pushedBackValue : justPushedBack) {
-      min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
-      pushedBack.add(pushedBackValue);
-    }
-    setPushedBackWatermark(min);
-    pushbackDoFnRunner.finishBundle();
-  }
-
-  @Override
-  public final void processElement2(
-      StreamRecord<RawUnionValue> streamRecord) throws Exception {
-    pushbackDoFnRunner.startBundle();
-
-    @SuppressWarnings("unchecked")
-    WindowedValue<Iterable<?>> value =
-        (WindowedValue<Iterable<?>>) streamRecord.getValue().getValue();
-
-    PCollectionView<?> sideInput = sideInputTagMapping.get(streamRecord.getValue().getUnionTag());
-    sideInputHandler.addSideInputValue(sideInput, value);
-
-    BagState<WindowedValue<InputT>> pushedBack =
-        pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
-
-    List<WindowedValue<InputT>> newPushedBack = new ArrayList<>();
-
-    Iterable<WindowedValue<InputT>> pushedBackContents = pushedBack.read();
-    if (pushedBackContents != null) {
-      for (WindowedValue<InputT> elem : pushedBackContents) {
-
-        // we need to set the correct key in case the operator is
-        // a (keyed) window operator
-        setKeyContextElement1(new StreamRecord<>(elem));
-
-        Iterable<WindowedValue<InputT>> justPushedBack =
-            pushbackDoFnRunner.processElementInReadyWindows(elem);
-        Iterables.addAll(newPushedBack, justPushedBack);
-      }
-    }
-
-    pushedBack.clear();
-    long min = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-    for (WindowedValue<InputT> pushedBackValue : newPushedBack) {
-      min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
-      pushedBack.add(pushedBackValue);
-    }
-    setPushedBackWatermark(min);
-
-    pushbackDoFnRunner.finishBundle();
-
-    // maybe output a new watermark
-    processWatermark1(new Watermark(currentInputWatermark));
-  }
-
-  @Override
-  public void processWatermark(Watermark mark) throws Exception {
-    processWatermark1(mark);
-  }
-
-  @Override
-  public void processWatermark1(Watermark mark) throws Exception {
-    if (keyCoder == null) {
-      this.currentInputWatermark = mark.getTimestamp();
-      long potentialOutputWatermark =
-          Math.min(getPushbackWatermarkHold(), currentInputWatermark);
-      if (potentialOutputWatermark > currentOutputWatermark) {
-        currentOutputWatermark = potentialOutputWatermark;
-        output.emitWatermark(new Watermark(currentOutputWatermark));
-      }
-    } else {
-      // fireTimers, so we need startBundle.
-      pushbackDoFnRunner.startBundle();
-
-      this.currentInputWatermark = mark.getTimestamp();
-
-      // hold back by the pushed back values waiting for side inputs
-      long actualInputWatermark = Math.min(getPushbackWatermarkHold(), mark.getTimestamp());
-
-      timerService.advanceWatermark(actualInputWatermark);
-
-      Instant watermarkHold = stateInternals.watermarkHold();
-
-      long combinedWatermarkHold = Math.min(watermarkHold.getMillis(), getPushbackWatermarkHold());
-
-      long potentialOutputWatermark = Math.min(currentInputWatermark, combinedWatermarkHold);
-
-      if (potentialOutputWatermark > currentOutputWatermark) {
-        currentOutputWatermark = potentialOutputWatermark;
-        output.emitWatermark(new Watermark(currentOutputWatermark));
-      }
-      pushbackDoFnRunner.finishBundle();
-    }
-  }
-
-  @Override
-  public void processWatermark2(Watermark mark) throws Exception {
-    // ignore watermarks from the side-input input
-  }
-
-  @Override
-  public void snapshotState(StateSnapshotContext context) throws Exception {
-    // copy from AbstractStreamOperator
-    if (getKeyedStateBackend() != null) {
-      KeyedStateCheckpointOutputStream out;
-
-      try {
-        out = context.getRawKeyedOperatorStateOutput();
-      } catch (Exception exception) {
-        throw new Exception("Could not open raw keyed operator state stream for "
-            + getOperatorName() + '.', exception);
-      }
-
-      try {
-        KeyGroupsList allKeyGroups = out.getKeyGroupList();
-        for (int keyGroupIdx : allKeyGroups) {
-          out.startNewKeyGroup(keyGroupIdx);
-
-          DataOutputViewStreamWrapper dov = new DataOutputViewStreamWrapper(out);
-
-          // if (this instanceof KeyGroupCheckpointedOperator)
-          snapshotKeyGroupState(keyGroupIdx, dov);
-
-          // We can't get all timerServices, so we just snapshot our timerService
-          // Maybe this is a normal DoFn that has no timerService
-          if (keyCoder != null) {
-            timerService.snapshotTimersForKeyGroup(dov, keyGroupIdx);
-          }
-
-        }
-      } catch (Exception exception) {
-        throw new Exception("Could not write timer service of " + getOperatorName()
-            + " to checkpoint state stream.", exception);
-      } finally {
-        try {
-          out.close();
-        } catch (Exception closeException) {
-          LOG.warn("Could not close raw keyed operator state stream for {}. This "
-              + "might have prevented deleting some state data.", getOperatorName(),
-              closeException);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void snapshotKeyGroupState(int keyGroupIndex, DataOutputStream out) throws Exception {
-    if (!sideInputs.isEmpty() && keyCoder != null) {
-      ((FlinkKeyGroupStateInternals) pushbackStateInternals).snapshotKeyGroupState(
-          keyGroupIndex, out);
-    }
-  }
-
-  @Override
-  public void initializeState(StateInitializationContext context) throws Exception {
-    if (getKeyedStateBackend() != null) {
-      int totalKeyGroups = getKeyedStateBackend().getNumberOfKeyGroups();
-      KeyGroupsList localKeyGroupRange = getKeyedStateBackend().getKeyGroupRange();
-
-      for (KeyGroupStatePartitionStreamProvider streamProvider : context.getRawKeyedStateInputs()) {
-        DataInputViewStreamWrapper div = new DataInputViewStreamWrapper(streamProvider.getStream());
-
-        int keyGroupIdx = streamProvider.getKeyGroupId();
-        checkArgument(localKeyGroupRange.contains(keyGroupIdx),
-            "Key Group " + keyGroupIdx + " does not belong to the local range.");
-
-        // if (this instanceof KeyGroupRestoringOperator)
-        restoreKeyGroupState(keyGroupIdx, div);
-
-        // We just initialize our timerService
-        if (keyCoder != null) {
-          if (timerService == null) {
-            timerService = new HeapInternalTimerService<>(
-                totalKeyGroups,
-                localKeyGroupRange,
-                this,
-                getRuntimeContext().getProcessingTimeService());
-          }
-          timerService.restoreTimersForKeyGroup(div, keyGroupIdx, getUserCodeClassloader());
-        }
-      }
-    }
-  }
-
-  @Override
-  public void restoreKeyGroupState(int keyGroupIndex, DataInputStream in) throws Exception {
-    if (!sideInputs.isEmpty() && keyCoder != null) {
-      if (pushbackStateInternals == null) {
-        pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder,
-            getKeyedStateBackend());
-      }
-      ((FlinkKeyGroupStateInternals) pushbackStateInternals)
-          .restoreKeyGroupState(keyGroupIndex, in, getUserCodeClassloader());
-    }
-  }
-
-  @Override
-  public void onEventTime(InternalTimer<Object, TimerData> timer) throws Exception {
-    fireTimer(timer);
-  }
-
-  @Override
-  public void onProcessingTime(InternalTimer<Object, TimerData> timer) throws Exception {
-    fireTimer(timer);
-  }
-
-  // allow overriding this in WindowDoFnOperator
-  public void fireTimer(InternalTimer<?, TimerData> timer) {
-    TimerInternals.TimerData timerData = timer.getNamespace();
-    StateNamespace namespace = timerData.getNamespace();
-    // This is a user timer, so namespace must be WindowNamespace
-    checkArgument(namespace instanceof WindowNamespace);
-    BoundedWindow window = ((WindowNamespace) namespace).getWindow();
-    pushbackDoFnRunner.onTimer(timerData.getTimerId(), window,
-        timerData.getTimestamp(), timerData.getDomain());
-  }
-
-  /**
-   * Factory for creating an {@link DoFnRunners.OutputManager} from
-   * a Flink {@link Output}.
-   */
-  interface OutputManagerFactory<OutputT> extends Serializable {
-    DoFnRunners.OutputManager create(Output<StreamRecord<OutputT>> output);
-  }
-
-  /**
-   * Default implementation of {@link OutputManagerFactory} that creates an
-   * {@link DoFnRunners.OutputManager} that only writes to
-   * a single logical output.
-   */
-  public static class DefaultOutputManagerFactory<OutputT>
-      implements OutputManagerFactory<OutputT> {
-    @Override
-    public DoFnRunners.OutputManager create(final Output<StreamRecord<OutputT>> output) {
-      return new DoFnRunners.OutputManager() {
-        @Override
-        public <T> void output(TupleTag<T> tag, WindowedValue<T> value) {
-          // with tagged outputs we can't get around this because we don't
-          // know our own output type...
-          @SuppressWarnings("unchecked")
-          OutputT castValue = (OutputT) value;
-          output.collect(new StreamRecord<>(castValue));
-        }
-      };
-    }
-  }
-
-  /**
-   * Implementation of {@link OutputManagerFactory} that creates an
-   * {@link DoFnRunners.OutputManager} that can write to multiple logical
-   * outputs by unioning them in a {@link RawUnionValue}.
-   */
-  public static class MultiOutputOutputManagerFactory
-      implements OutputManagerFactory<RawUnionValue> {
-
-    Map<TupleTag<?>, Integer> mapping;
-
-    public MultiOutputOutputManagerFactory(Map<TupleTag<?>, Integer> mapping) {
-      this.mapping = mapping;
-    }
-
-    @Override
-    public DoFnRunners.OutputManager create(final Output<StreamRecord<RawUnionValue>> output) {
-      return new DoFnRunners.OutputManager() {
-        @Override
-        public <T> void output(TupleTag<T> tag, WindowedValue<T> value) {
-          int intTag = mapping.get(tag);
-          output.collect(new StreamRecord<>(new RawUnionValue(intTag, value)));
-        }
-      };
-    }
-  }
-
-  /**
-   * {@link StepContext} for running {@link DoFn DoFns} on Flink. This does not allow
-   * accessing state or timer internals.
-   */
-  protected class StepContext implements ExecutionContext.StepContext {
-
-    @Override
-    public String getStepName() {
-      return null;
-    }
-
-    @Override
-    public String getTransformName() {
-      return null;
-    }
-
-    @Override
-    public void noteOutput(WindowedValue<?> output) {}
-
-    @Override
-    public void noteOutput(TupleTag<?> tag, WindowedValue<?> output) {}
-
-    @Override
-    public <T, W extends BoundedWindow> void writePCollectionViewData(
-        TupleTag<?> tag,
-        Iterable<WindowedValue<T>> data,
-        Coder<Iterable<WindowedValue<T>>> dataCoder,
-        W window,
-        Coder<W> windowCoder) throws IOException {
-      throw new UnsupportedOperationException("Writing side-input data is not supported.");
-    }
-
-    @Override
-    public StateInternals<?> stateInternals() {
-      return stateInternals;
-    }
-
-    @Override
-    public TimerInternals timerInternals() {
-      return timerInternals;
-    }
-  }
-
-  private class FlinkTimerInternals implements TimerInternals {
-
-    @Override
-    public void setTimer(
-        StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
-      setTimer(TimerData.of(timerId, namespace, target, timeDomain));
-    }
-
-    @Deprecated
-    @Override
-    public void setTimer(TimerData timerKey) {
-      long time = timerKey.getTimestamp().getMillis();
-      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
-        timerService.registerEventTimeTimer(timerKey, time);
-      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
-        timerService.registerProcessingTimeTimer(timerKey, time);
-      } else {
-        throw new UnsupportedOperationException(
-            "Unsupported time domain: " + timerKey.getDomain());
-      }
-    }
-
-    @Deprecated
-    @Override
-    public void deleteTimer(StateNamespace namespace, String timerId) {
-      throw new UnsupportedOperationException(
-          "Canceling of a timer by ID is not yet supported.");
-    }
-
-    @Override
-    public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
-      throw new UnsupportedOperationException(
-          "Canceling of a timer by ID is not yet supported.");
-    }
-
-    @Deprecated
-    @Override
-    public void deleteTimer(TimerData timerKey) {
-      long time = timerKey.getTimestamp().getMillis();
-      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
-        timerService.deleteEventTimeTimer(timerKey, time);
-      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
-        timerService.deleteProcessingTimeTimer(timerKey, time);
-      } else {
-        throw new UnsupportedOperationException(
-            "Unsupported time domain: " + timerKey.getDomain());
-      }
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return new Instant(timerService.currentProcessingTime());
-    }
-
-    @Nullable
-    @Override
-    public Instant currentSynchronizedProcessingTime() {
-      return new Instant(timerService.currentProcessingTime());
-    }
-
-    @Override
-    public Instant currentInputWatermarkTime() {
-      return new Instant(Math.min(currentInputWatermark, getPushbackWatermarkHold()));
-    }
-
-    @Nullable
-    @Override
-    public Instant currentOutputWatermarkTime() {
-      return new Instant(currentOutputWatermark);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java
deleted file mode 100644
index dce2e68..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import java.nio.ByteBuffer;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.util.CoderUtils;
-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.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-
-/**
- * {@link KeySelector} that retrieves a key from a {@link KV}. This will return
- * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures
- * that all key comparisons/hashing happen on the encoded form.
- */
-public class KvToByteBufferKeySelector<K, V>
-    implements KeySelector<WindowedValue<KV<K, V>>, ByteBuffer>,
-    ResultTypeQueryable<ByteBuffer> {
-
-  private final Coder<K> keyCoder;
-
-  public KvToByteBufferKeySelector(Coder<K> keyCoder) {
-    this.keyCoder = keyCoder;
-  }
-
-  @Override
-  public ByteBuffer getKey(WindowedValue<KV<K, V>> value) throws Exception {
-    K key = value.getValue().getKey();
-    byte[] keyBytes = CoderUtils.encodeToByteArray(keyCoder, key);
-    return ByteBuffer.wrap(keyBytes);
-  }
-
-  @Override
-  public TypeInformation<ByteBuffer> getProducedType() {
-    return new GenericTypeInfo<>(ByteBuffer.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
deleted file mode 100644
index e843660..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import java.util.Collections;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * Singleton keyed word item.
- */
-public class SingletonKeyedWorkItem<K, ElemT> implements KeyedWorkItem<K, ElemT> {
-
-  final K key;
-  final WindowedValue<ElemT> value;
-
-  public SingletonKeyedWorkItem(K key, WindowedValue<ElemT> value) {
-    this.key = key;
-    this.value = value;
-  }
-
-  @Override
-  public K key() {
-    return key;
-  }
-
-  public WindowedValue<ElemT> value() {
-    return value;
-  }
-
-  @Override
-  public Iterable<TimerInternals.TimerData> timersIterable() {
-    return Collections.EMPTY_LIST;
-  }
-
-  @Override
-  public Iterable<WindowedValue<ElemT>> elementsIterable() {
-    return Collections.singletonList(value);
-  }
-}


[05/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
new file mode 100644
index 0000000..e24bf31
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.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 org.apache.beam.runners.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+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;
+
+/**
+ * Flink {@link TypeInformation} for Beam values that have been encoded to byte data
+ * by a {@link Coder}.
+ */
+public class EncodedValueTypeInformation
+    extends TypeInformation<byte[]>
+    implements AtomicType<byte[]> {
+
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  public boolean isBasicType() {
+    return false;
+  }
+
+  @Override
+  public boolean isTupleType() {
+    return false;
+  }
+
+  @Override
+  public int getArity() {
+    return 0;
+  }
+
+  @Override
+  public int getTotalFields() {
+    return 0;
+  }
+
+  @Override
+  public Class<byte[]> getTypeClass() {
+    return byte[].class;
+  }
+
+  @Override
+  public boolean isKeyType() {
+    return true;
+  }
+
+  @Override
+  public TypeSerializer<byte[]> createSerializer(ExecutionConfig executionConfig) {
+    return new EncodedValueSerializer();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return other instanceof EncodedValueTypeInformation;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.getClass().hashCode();
+  }
+
+  @Override
+  public boolean canEqual(Object obj) {
+    return obj instanceof EncodedValueTypeInformation;
+  }
+
+  @Override
+  public String toString() {
+    return "EncodedValueTypeInformation";
+  }
+
+  @Override
+  public TypeComparator<byte[]> createComparator(
+      boolean sortOrderAscending,
+      ExecutionConfig executionConfig) {
+    return new EncodedValueComparator(sortOrderAscending);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
new file mode 100644
index 0000000..36b5ba3
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.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 org.apache.beam.runners.flink.translation.types;
+
+import java.io.ByteArrayOutputStream;
+
+/**
+ * Version of {@link java.io.ByteArrayOutputStream} that allows to retrieve the internal
+ * byte[] buffer without incurring an array copy.
+ */
+public class InspectableByteArrayOutputStream extends ByteArrayOutputStream {
+
+  /**
+   * Get the underlying byte array.
+   */
+  public byte[] getBuffer() {
+    return buf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
new file mode 100644
index 0000000..9df6836
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.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 org.apache.beam.runners.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+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;
+
+/**
+ * {@link KeySelector} that extracts the key from a {@link KV} and returns
+ * it in encoded form as a {@code byte} array.
+ */
+public class KvKeySelector<InputT, K>
+    implements KeySelector<WindowedValue<KV<K, InputT>>, byte[]>, ResultTypeQueryable<byte[]> {
+
+  private final Coder<K> keyCoder;
+
+  public KvKeySelector(Coder<K> keyCoder) {
+    this.keyCoder = keyCoder;
+  }
+
+  @Override
+  public byte[] getKey(WindowedValue<KV<K, InputT>> value) throws Exception {
+    return CoderUtils.encodeToByteArray(keyCoder, value.getValue().getKey());
+  }
+
+  @Override
+  public TypeInformation<byte[]> getProducedType() {
+    return new EncodedValueTypeInformation();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
new file mode 100644
index 0000000..6fb3182
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.types;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
new file mode 100644
index 0000000..2256bb1
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.flink.translation.utils;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.IOChannelUtils;
+
+/**
+ * Encapsulates the PipelineOptions in serialized form to ship them to the cluster.
+ */
+public class SerializedPipelineOptions implements Serializable {
+
+  private final byte[] serializedOptions;
+
+  /** Lazily initialized copy of deserialized options. */
+  private transient PipelineOptions pipelineOptions;
+
+  public SerializedPipelineOptions(PipelineOptions options) {
+    checkNotNull(options, "PipelineOptions must not be null.");
+
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+      new ObjectMapper().writeValue(baos, options);
+      this.serializedOptions = baos.toByteArray();
+    } catch (Exception e) {
+      throw new RuntimeException("Couldn't serialize PipelineOptions.", e);
+    }
+
+  }
+
+  public PipelineOptions getPipelineOptions() {
+    if (pipelineOptions == null) {
+      try {
+        pipelineOptions = new ObjectMapper().readValue(serializedOptions, PipelineOptions.class);
+
+        IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions);
+        FileSystems.setDefaultConfigInWorkers(pipelineOptions);
+      } catch (IOException e) {
+        throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e);
+      }
+    }
+
+    return pipelineOptions;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
new file mode 100644
index 0000000..5dedd53
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
new file mode 100644
index 0000000..82a2c4e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import org.apache.flink.core.memory.DataInputView;
+
+/**
+ * Wrapper for {@link DataInputView}. We need this because Flink reads data using a
+ * {@link org.apache.flink.core.memory.DataInputView} while
+ * Dataflow {@link org.apache.beam.sdk.coders.Coder}s expect an
+ * {@link java.io.InputStream}.
+ */
+public class DataInputViewWrapper extends InputStream {
+
+  private DataInputView inputView;
+
+  public DataInputViewWrapper(DataInputView inputView) {
+    this.inputView = inputView;
+  }
+
+  public void setInputView(DataInputView inputView) {
+    this.inputView = inputView;
+  }
+
+  @Override
+  public int read() throws IOException {
+    try {
+      return inputView.readUnsignedByte();
+    } catch (EOFException e) {
+      // translate between DataInput and InputStream,
+      // DataInput signals EOF by exception, InputStream does it by returning -1
+      return -1;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return inputView.read(b, off, len);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
new file mode 100644
index 0000000..f2d9db2
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.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 org.apache.beam.runners.flink.translation.wrappers;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.flink.core.memory.DataOutputView;
+
+/**
+ * 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 org.apache.beam.sdk.coders.Coder}s expect an
+ * {@link java.io.OutputStream}.
+ */
+public class DataOutputViewWrapper extends OutputStream {
+
+  private DataOutputView outputView;
+
+  public DataOutputViewWrapper(DataOutputView outputView) {
+    this.outputView = outputView;
+  }
+
+  public void setOutputView(DataOutputView outputView) {
+    this.outputView = outputView;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    outputView.write(b);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    outputView.write(b, off, len);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
new file mode 100644
index 0000000..70d97e3
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.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 org.apache.beam.runners.flink.translation.wrappers;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import java.io.Serializable;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.flink.api.common.accumulators.Accumulator;
+
+/**
+ * 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 org.apache.beam.sdk.transforms.ParDo}
+ * operation.
+ */
+public class SerializableFnAggregatorWrapper<InputT, OutputT>
+    implements Aggregator<InputT, OutputT>, Accumulator<InputT, Serializable> {
+
+  private OutputT aa;
+  private Combine.CombineFn<InputT, ?, OutputT> combiner;
+
+  public SerializableFnAggregatorWrapper(Combine.CombineFn<InputT, ?, OutputT> combiner) {
+    this.combiner = combiner;
+    resetLocal();
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void add(InputT value) {
+    this.aa = combiner.apply(ImmutableList.of((InputT) aa, value));
+  }
+
+  @Override
+  public Serializable getLocalValue() {
+    return (Serializable) aa;
+  }
+
+  @Override
+  public void resetLocal() {
+    this.aa = combiner.apply(ImmutableList.<InputT>of());
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void merge(Accumulator<InputT, Serializable> other) {
+    this.aa = combiner.apply(ImmutableList.of((InputT) aa, (InputT) other.getLocalValue()));
+  }
+
+  @Override
+  public void addValue(InputT value) {
+    add(value);
+  }
+
+  @Override
+  public String getName() {
+    return "Aggregator :" + combiner.toString();
+  }
+
+  @Override
+  public Combine.CombineFn<InputT, ?, OutputT> getCombineFn() {
+    return combiner;
+  }
+
+  @Override
+  public Accumulator<InputT, Serializable> clone() {
+    try {
+      super.clone();
+    } catch (CloneNotSupportedException e) {
+      // Flink Accumulators cannot throw CloneNotSupportedException, work around that.
+      throw new RuntimeException(e);
+    }
+
+    // copy it by merging
+    OutputT resultCopy = combiner.apply(Lists.newArrayList((InputT) aa));
+    SerializableFnAggregatorWrapper<InputT, OutputT> result = new
+        SerializableFnAggregatorWrapper<>(combiner);
+
+    result.aa = resultCopy;
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
new file mode 100644
index 0000000..a87472b
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+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.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}.
+ */
+public class SourceInputFormat<T>
+    implements InputFormat<WindowedValue<T>, SourceInputSplit<T>> {
+  private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
+
+  private final BoundedSource<T> initialSource;
+
+  private transient PipelineOptions options;
+  private final SerializedPipelineOptions serializedOptions;
+
+  private transient BoundedSource.BoundedReader<T> reader;
+  private boolean inputAvailable = false;
+
+  public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
+    this.initialSource = initialSource;
+    this.serializedOptions = new SerializedPipelineOptions(options);
+  }
+
+  @Override
+  public void configure(Configuration configuration) {
+    options = serializedOptions.getPipelineOptions();
+  }
+
+  @Override
+  public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
+    reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
+    inputAvailable = reader.start();
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
+    try {
+      final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
+
+      return new BaseStatistics() {
+        @Override
+        public long getTotalInputSize() {
+          return estimatedSize;
+        }
+
+        @Override
+        public long getNumberOfRecords() {
+          return BaseStatistics.NUM_RECORDS_UNKNOWN;
+        }
+
+        @Override
+        public float getAverageRecordWidth() {
+          return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
+        }
+      };
+    } catch (Exception e) {
+      LOG.warn("Could not read Source statistics: {}", e);
+    }
+
+    return null;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
+    try {
+      long desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
+      List<? extends Source<T>> shards =
+          initialSource.split(desiredSizeBytes, options);
+      int numShards = shards.size();
+      SourceInputSplit<T>[] sourceInputSplits = new SourceInputSplit[numShards];
+      for (int i = 0; i < numShards; i++) {
+        sourceInputSplits[i] = new SourceInputSplit<>(shards.get(i), i);
+      }
+      return sourceInputSplits;
+    } catch (Exception e) {
+      throw new IOException("Could not create input splits from Source.", e);
+    }
+  }
+
+  @Override
+  public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
+    return new DefaultInputSplitAssigner(sourceInputSplits);
+  }
+
+
+  @Override
+  public boolean reachedEnd() throws IOException {
+    return !inputAvailable;
+  }
+
+  @Override
+  public WindowedValue<T> nextRecord(WindowedValue<T> t) throws IOException {
+    if (inputAvailable) {
+      final T current = reader.getCurrent();
+      final Instant timestamp = reader.getCurrentTimestamp();
+      // advance reader to have a record ready next time
+      inputAvailable = reader.advance();
+      return WindowedValue.of(
+          current,
+          timestamp,
+          GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+    }
+
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    // TODO null check can be removed once FLINK-3796 is fixed
+    if (reader != null) {
+      reader.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
new file mode 100644
index 0000000..e4a7386
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers;
+
+import org.apache.beam.sdk.io.Source;
+import org.apache.flink.core.io.InputSplit;
+
+/**
+ * {@link org.apache.flink.core.io.InputSplit} for
+ * {@link org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat}. We pass
+ * the sharded Source around in the input split because Sources simply split up into several
+ * Sources for sharding. This is different to how Flink creates a separate InputSplit from
+ * an InputFormat.
+ */
+public class SourceInputSplit<T> implements InputSplit {
+
+  private Source<T> source;
+  private int splitNumber;
+
+  public SourceInputSplit() {
+  }
+
+  public SourceInputSplit(Source<T> source, int splitNumber) {
+    this.source = source;
+    this.splitNumber = splitNumber;
+  }
+
+  @Override
+  public int getSplitNumber() {
+    return splitNumber;
+  }
+
+  public Source<T> getSource() {
+    return source;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
new file mode 100644
index 0000000..72f7deb
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
new file mode 100644
index 0000000..8a09286
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -0,0 +1,774 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Iterables;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.AggregatorFactory;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.ExecutionContext;
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
+import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
+import org.apache.beam.runners.core.SideInputHandler;
+import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateNamespaces;
+import org.apache.beam.runners.core.StateNamespaces.WindowNamespace;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTags;
+import org.apache.beam.runners.core.StatefulDoFnRunner;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.core.TimerInternals.TimerData;
+import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.KeyGroupCheckpointedOperator;
+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.join.RawUnionValue;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.NullSideInputReader;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.HeapInternalTimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.joda.time.Instant;
+
+/**
+ * Flink operator for executing {@link DoFn DoFns}.
+ *
+ * @param <InputT> the input type of the {@link DoFn}
+ * @param <FnOutputT> the output type of the {@link DoFn}
+ * @param <OutputT> the output type of the operator, this can be different from the fn output
+ *                 type when we have additional tagged outputs
+ */
+public class DoFnOperator<InputT, FnOutputT, OutputT>
+    extends AbstractStreamOperator<OutputT>
+    implements OneInputStreamOperator<WindowedValue<InputT>, OutputT>,
+      TwoInputStreamOperator<WindowedValue<InputT>, RawUnionValue, OutputT>,
+    KeyGroupCheckpointedOperator, Triggerable<Object, TimerData> {
+
+  protected DoFn<InputT, FnOutputT> doFn;
+
+  protected final SerializedPipelineOptions serializedOptions;
+
+  protected final TupleTag<FnOutputT> mainOutputTag;
+  protected final List<TupleTag<?>> additionalOutputTags;
+
+  protected final Collection<PCollectionView<?>> sideInputs;
+  protected final Map<Integer, PCollectionView<?>> sideInputTagMapping;
+
+  protected final WindowingStrategy<?, ?> windowingStrategy;
+
+  protected final OutputManagerFactory<OutputT> outputManagerFactory;
+
+  protected transient DoFnRunner<InputT, FnOutputT> doFnRunner;
+  protected transient PushbackSideInputDoFnRunner<InputT, FnOutputT> pushbackDoFnRunner;
+
+  protected transient SideInputHandler sideInputHandler;
+
+  protected transient SideInputReader sideInputReader;
+
+  protected transient DoFnRunners.OutputManager outputManager;
+
+  private transient DoFnInvoker<InputT, FnOutputT> doFnInvoker;
+
+  protected transient long currentInputWatermark;
+
+  protected transient long currentOutputWatermark;
+
+  private transient StateTag<Object, BagState<WindowedValue<InputT>>> pushedBackTag;
+
+  protected transient FlinkStateInternals<?> stateInternals;
+
+  private Coder<WindowedValue<InputT>> inputCoder;
+
+  private final Coder<?> keyCoder;
+
+  private final TimerInternals.TimerDataCoder timerCoder;
+
+  protected transient HeapInternalTimerService<?, TimerInternals.TimerData> timerService;
+
+  protected transient FlinkTimerInternals timerInternals;
+
+  private transient StateInternals<?> pushbackStateInternals;
+
+  private transient Optional<Long> pushedBackWatermark;
+
+  public DoFnOperator(
+      DoFn<InputT, FnOutputT> doFn,
+      Coder<WindowedValue<InputT>> inputCoder,
+      TupleTag<FnOutputT> mainOutputTag,
+      List<TupleTag<?>> additionalOutputTags,
+      OutputManagerFactory<OutputT> outputManagerFactory,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Map<Integer, PCollectionView<?>> sideInputTagMapping,
+      Collection<PCollectionView<?>> sideInputs,
+      PipelineOptions options,
+      Coder<?> keyCoder) {
+    this.doFn = doFn;
+    this.inputCoder = inputCoder;
+    this.mainOutputTag = mainOutputTag;
+    this.additionalOutputTags = additionalOutputTags;
+    this.sideInputTagMapping = sideInputTagMapping;
+    this.sideInputs = sideInputs;
+    this.serializedOptions = new SerializedPipelineOptions(options);
+    this.windowingStrategy = windowingStrategy;
+    this.outputManagerFactory = outputManagerFactory;
+
+    setChainingStrategy(ChainingStrategy.ALWAYS);
+
+    this.keyCoder = keyCoder;
+
+    this.timerCoder =
+        TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
+  }
+
+  private ExecutionContext.StepContext createStepContext() {
+    return new StepContext();
+  }
+
+  // allow overriding this in WindowDoFnOperator because this one dynamically creates
+  // the DoFn
+  protected DoFn<InputT, FnOutputT> getDoFn() {
+    return doFn;
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    currentInputWatermark = Long.MIN_VALUE;
+    currentOutputWatermark = Long.MIN_VALUE;
+
+    AggregatorFactory aggregatorFactory = new AggregatorFactory() {
+      @Override
+      public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
+          Class<?> fnClass,
+          ExecutionContext.StepContext stepContext,
+          String aggregatorName,
+          Combine.CombineFn<InputT, AccumT, OutputT> combine) {
+
+        @SuppressWarnings("unchecked")
+        SerializableFnAggregatorWrapper<InputT, OutputT> result =
+            (SerializableFnAggregatorWrapper<InputT, OutputT>)
+                getRuntimeContext().getAccumulator(aggregatorName);
+
+        if (result == null) {
+          result = new SerializableFnAggregatorWrapper<>(combine);
+          getRuntimeContext().addAccumulator(aggregatorName, result);
+        }
+        return result;
+      }
+    };
+
+    sideInputReader = NullSideInputReader.of(sideInputs);
+
+    if (!sideInputs.isEmpty()) {
+
+      pushedBackTag = StateTags.bag("pushed-back-values", inputCoder);
+
+      FlinkBroadcastStateInternals sideInputStateInternals =
+          new FlinkBroadcastStateInternals<>(
+              getContainingTask().getIndexInSubtaskGroup(), getOperatorStateBackend());
+
+      sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals);
+      sideInputReader = sideInputHandler;
+
+      // maybe init by initializeState
+      if (pushbackStateInternals == null) {
+        if (keyCoder != null) {
+          pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder,
+              getKeyedStateBackend());
+        } else {
+          pushbackStateInternals =
+              new FlinkSplitStateInternals<Object>(getOperatorStateBackend());
+        }
+      }
+
+      pushedBackWatermark = Optional.absent();
+
+    }
+
+    outputManager = outputManagerFactory.create(output);
+
+    // StatefulPardo or WindowDoFn
+    if (keyCoder != null) {
+      stateInternals = new FlinkStateInternals<>((KeyedStateBackend) getKeyedStateBackend(),
+          keyCoder);
+
+      timerService = (HeapInternalTimerService<?, TimerInternals.TimerData>)
+          getInternalTimerService("beam-timer", new CoderTypeSerializer<>(timerCoder), this);
+
+      timerInternals = new FlinkTimerInternals();
+
+    }
+
+    // WindowDoFnOperator need use state and timer to get DoFn.
+    // So must wait StateInternals and TimerInternals ready.
+    this.doFn = getDoFn();
+    doFnInvoker = DoFnInvokers.invokerFor(doFn);
+
+    doFnInvoker.invokeSetup();
+
+    ExecutionContext.StepContext stepContext = createStepContext();
+
+    doFnRunner = DoFnRunners.simpleRunner(
+        serializedOptions.getPipelineOptions(),
+        doFn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        additionalOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy);
+
+    if (doFn instanceof GroupAlsoByWindowViaWindowSetNewDoFn) {
+      // When the doFn is this, we know it came from WindowDoFnOperator and
+      //   InputT = KeyedWorkItem<K, V>
+      //   OutputT = KV<K, V>
+      //
+      // for some K, V
+
+
+      doFnRunner = DoFnRunners.lateDataDroppingRunner(
+          (DoFnRunner) doFnRunner,
+          stepContext,
+          windowingStrategy,
+          ((GroupAlsoByWindowViaWindowSetNewDoFn) doFn).getDroppedDueToLatenessAggregator());
+    } else if (keyCoder != null) {
+      // It is a stateful DoFn
+
+      StatefulDoFnRunner.CleanupTimer cleanupTimer =
+          new StatefulDoFnRunner.TimeInternalsCleanupTimer(
+              stepContext.timerInternals(), windowingStrategy);
+
+      // we don't know the window type
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder windowCoder = windowingStrategy.getWindowFn().windowCoder();
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      StatefulDoFnRunner.StateCleaner<?> stateCleaner =
+          new StatefulDoFnRunner.StateInternalsStateCleaner<>(
+              doFn, stepContext.stateInternals(), windowCoder);
+
+      doFnRunner = DoFnRunners.defaultStatefulDoFnRunner(
+          doFn,
+          doFnRunner,
+          stepContext,
+          aggregatorFactory,
+          windowingStrategy,
+          cleanupTimer,
+          stateCleaner);
+    }
+
+    pushbackDoFnRunner =
+        SimplePushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler);
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    doFnInvoker.invokeTeardown();
+  }
+
+  protected final long getPushbackWatermarkHold() {
+    // if we don't have side inputs we never hold the watermark
+    if (sideInputs.isEmpty()) {
+      return BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+    }
+
+    try {
+      checkInitPushedBackWatermark();
+      return pushedBackWatermark.get();
+    } catch (Exception e) {
+      throw new RuntimeException("Error retrieving pushed back watermark state.", e);
+    }
+  }
+
+  private void checkInitPushedBackWatermark() {
+    // init and restore from pushedBack state.
+    // Not done in initializeState, because OperatorState is not ready.
+    if (!pushedBackWatermark.isPresent()) {
+
+      BagState<WindowedValue<InputT>> pushedBack =
+          pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
+
+      long min = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+      for (WindowedValue<InputT> value : pushedBack.read()) {
+        min = Math.min(min, value.getTimestamp().getMillis());
+      }
+      setPushedBackWatermark(min);
+    }
+  }
+
+  @Override
+  public final void processElement(
+      StreamRecord<WindowedValue<InputT>> streamRecord) throws Exception {
+    doFnRunner.startBundle();
+    doFnRunner.processElement(streamRecord.getValue());
+    doFnRunner.finishBundle();
+  }
+
+  private void setPushedBackWatermark(long watermark) {
+    pushedBackWatermark = Optional.fromNullable(watermark);
+  }
+
+  @Override
+  public final void processElement1(
+      StreamRecord<WindowedValue<InputT>> streamRecord) throws Exception {
+    pushbackDoFnRunner.startBundle();
+    Iterable<WindowedValue<InputT>> justPushedBack =
+        pushbackDoFnRunner.processElementInReadyWindows(streamRecord.getValue());
+
+    BagState<WindowedValue<InputT>> pushedBack =
+        pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
+
+    checkInitPushedBackWatermark();
+
+    long min = pushedBackWatermark.get();
+    for (WindowedValue<InputT> pushedBackValue : justPushedBack) {
+      min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
+      pushedBack.add(pushedBackValue);
+    }
+    setPushedBackWatermark(min);
+    pushbackDoFnRunner.finishBundle();
+  }
+
+  @Override
+  public final void processElement2(
+      StreamRecord<RawUnionValue> streamRecord) throws Exception {
+    pushbackDoFnRunner.startBundle();
+
+    @SuppressWarnings("unchecked")
+    WindowedValue<Iterable<?>> value =
+        (WindowedValue<Iterable<?>>) streamRecord.getValue().getValue();
+
+    PCollectionView<?> sideInput = sideInputTagMapping.get(streamRecord.getValue().getUnionTag());
+    sideInputHandler.addSideInputValue(sideInput, value);
+
+    BagState<WindowedValue<InputT>> pushedBack =
+        pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag);
+
+    List<WindowedValue<InputT>> newPushedBack = new ArrayList<>();
+
+    Iterable<WindowedValue<InputT>> pushedBackContents = pushedBack.read();
+    if (pushedBackContents != null) {
+      for (WindowedValue<InputT> elem : pushedBackContents) {
+
+        // we need to set the correct key in case the operator is
+        // a (keyed) window operator
+        setKeyContextElement1(new StreamRecord<>(elem));
+
+        Iterable<WindowedValue<InputT>> justPushedBack =
+            pushbackDoFnRunner.processElementInReadyWindows(elem);
+        Iterables.addAll(newPushedBack, justPushedBack);
+      }
+    }
+
+    pushedBack.clear();
+    long min = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+    for (WindowedValue<InputT> pushedBackValue : newPushedBack) {
+      min = Math.min(min, pushedBackValue.getTimestamp().getMillis());
+      pushedBack.add(pushedBackValue);
+    }
+    setPushedBackWatermark(min);
+
+    pushbackDoFnRunner.finishBundle();
+
+    // maybe output a new watermark
+    processWatermark1(new Watermark(currentInputWatermark));
+  }
+
+  @Override
+  public void processWatermark(Watermark mark) throws Exception {
+    processWatermark1(mark);
+  }
+
+  @Override
+  public void processWatermark1(Watermark mark) throws Exception {
+    if (keyCoder == null) {
+      this.currentInputWatermark = mark.getTimestamp();
+      long potentialOutputWatermark =
+          Math.min(getPushbackWatermarkHold(), currentInputWatermark);
+      if (potentialOutputWatermark > currentOutputWatermark) {
+        currentOutputWatermark = potentialOutputWatermark;
+        output.emitWatermark(new Watermark(currentOutputWatermark));
+      }
+    } else {
+      // fireTimers, so we need startBundle.
+      pushbackDoFnRunner.startBundle();
+
+      this.currentInputWatermark = mark.getTimestamp();
+
+      // hold back by the pushed back values waiting for side inputs
+      long actualInputWatermark = Math.min(getPushbackWatermarkHold(), mark.getTimestamp());
+
+      timerService.advanceWatermark(actualInputWatermark);
+
+      Instant watermarkHold = stateInternals.watermarkHold();
+
+      long combinedWatermarkHold = Math.min(watermarkHold.getMillis(), getPushbackWatermarkHold());
+
+      long potentialOutputWatermark = Math.min(currentInputWatermark, combinedWatermarkHold);
+
+      if (potentialOutputWatermark > currentOutputWatermark) {
+        currentOutputWatermark = potentialOutputWatermark;
+        output.emitWatermark(new Watermark(currentOutputWatermark));
+      }
+      pushbackDoFnRunner.finishBundle();
+    }
+  }
+
+  @Override
+  public void processWatermark2(Watermark mark) throws Exception {
+    // ignore watermarks from the side-input input
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    // copy from AbstractStreamOperator
+    if (getKeyedStateBackend() != null) {
+      KeyedStateCheckpointOutputStream out;
+
+      try {
+        out = context.getRawKeyedOperatorStateOutput();
+      } catch (Exception exception) {
+        throw new Exception("Could not open raw keyed operator state stream for "
+            + getOperatorName() + '.', exception);
+      }
+
+      try {
+        KeyGroupsList allKeyGroups = out.getKeyGroupList();
+        for (int keyGroupIdx : allKeyGroups) {
+          out.startNewKeyGroup(keyGroupIdx);
+
+          DataOutputViewStreamWrapper dov = new DataOutputViewStreamWrapper(out);
+
+          // if (this instanceof KeyGroupCheckpointedOperator)
+          snapshotKeyGroupState(keyGroupIdx, dov);
+
+          // We can't get all timerServices, so we just snapshot our timerService
+          // Maybe this is a normal DoFn that has no timerService
+          if (keyCoder != null) {
+            timerService.snapshotTimersForKeyGroup(dov, keyGroupIdx);
+          }
+
+        }
+      } catch (Exception exception) {
+        throw new Exception("Could not write timer service of " + getOperatorName()
+            + " to checkpoint state stream.", exception);
+      } finally {
+        try {
+          out.close();
+        } catch (Exception closeException) {
+          LOG.warn("Could not close raw keyed operator state stream for {}. This "
+              + "might have prevented deleting some state data.", getOperatorName(),
+              closeException);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void snapshotKeyGroupState(int keyGroupIndex, DataOutputStream out) throws Exception {
+    if (!sideInputs.isEmpty() && keyCoder != null) {
+      ((FlinkKeyGroupStateInternals) pushbackStateInternals).snapshotKeyGroupState(
+          keyGroupIndex, out);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    if (getKeyedStateBackend() != null) {
+      int totalKeyGroups = getKeyedStateBackend().getNumberOfKeyGroups();
+      KeyGroupsList localKeyGroupRange = getKeyedStateBackend().getKeyGroupRange();
+
+      for (KeyGroupStatePartitionStreamProvider streamProvider : context.getRawKeyedStateInputs()) {
+        DataInputViewStreamWrapper div = new DataInputViewStreamWrapper(streamProvider.getStream());
+
+        int keyGroupIdx = streamProvider.getKeyGroupId();
+        checkArgument(localKeyGroupRange.contains(keyGroupIdx),
+            "Key Group " + keyGroupIdx + " does not belong to the local range.");
+
+        // if (this instanceof KeyGroupRestoringOperator)
+        restoreKeyGroupState(keyGroupIdx, div);
+
+        // We just initialize our timerService
+        if (keyCoder != null) {
+          if (timerService == null) {
+            timerService = new HeapInternalTimerService<>(
+                totalKeyGroups,
+                localKeyGroupRange,
+                this,
+                getRuntimeContext().getProcessingTimeService());
+          }
+          timerService.restoreTimersForKeyGroup(div, keyGroupIdx, getUserCodeClassloader());
+        }
+      }
+    }
+  }
+
+  @Override
+  public void restoreKeyGroupState(int keyGroupIndex, DataInputStream in) throws Exception {
+    if (!sideInputs.isEmpty() && keyCoder != null) {
+      if (pushbackStateInternals == null) {
+        pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder,
+            getKeyedStateBackend());
+      }
+      ((FlinkKeyGroupStateInternals) pushbackStateInternals)
+          .restoreKeyGroupState(keyGroupIndex, in, getUserCodeClassloader());
+    }
+  }
+
+  @Override
+  public void onEventTime(InternalTimer<Object, TimerData> timer) throws Exception {
+    fireTimer(timer);
+  }
+
+  @Override
+  public void onProcessingTime(InternalTimer<Object, TimerData> timer) throws Exception {
+    fireTimer(timer);
+  }
+
+  // allow overriding this in WindowDoFnOperator
+  public void fireTimer(InternalTimer<?, TimerData> timer) {
+    TimerInternals.TimerData timerData = timer.getNamespace();
+    StateNamespace namespace = timerData.getNamespace();
+    // This is a user timer, so namespace must be WindowNamespace
+    checkArgument(namespace instanceof WindowNamespace);
+    BoundedWindow window = ((WindowNamespace) namespace).getWindow();
+    pushbackDoFnRunner.onTimer(timerData.getTimerId(), window,
+        timerData.getTimestamp(), timerData.getDomain());
+  }
+
+  /**
+   * Factory for creating an {@link DoFnRunners.OutputManager} from
+   * a Flink {@link Output}.
+   */
+  interface OutputManagerFactory<OutputT> extends Serializable {
+    DoFnRunners.OutputManager create(Output<StreamRecord<OutputT>> output);
+  }
+
+  /**
+   * Default implementation of {@link OutputManagerFactory} that creates an
+   * {@link DoFnRunners.OutputManager} that only writes to
+   * a single logical output.
+   */
+  public static class DefaultOutputManagerFactory<OutputT>
+      implements OutputManagerFactory<OutputT> {
+    @Override
+    public DoFnRunners.OutputManager create(final Output<StreamRecord<OutputT>> output) {
+      return new DoFnRunners.OutputManager() {
+        @Override
+        public <T> void output(TupleTag<T> tag, WindowedValue<T> value) {
+          // with tagged outputs we can't get around this because we don't
+          // know our own output type...
+          @SuppressWarnings("unchecked")
+          OutputT castValue = (OutputT) value;
+          output.collect(new StreamRecord<>(castValue));
+        }
+      };
+    }
+  }
+
+  /**
+   * Implementation of {@link OutputManagerFactory} that creates an
+   * {@link DoFnRunners.OutputManager} that can write to multiple logical
+   * outputs by unioning them in a {@link RawUnionValue}.
+   */
+  public static class MultiOutputOutputManagerFactory
+      implements OutputManagerFactory<RawUnionValue> {
+
+    Map<TupleTag<?>, Integer> mapping;
+
+    public MultiOutputOutputManagerFactory(Map<TupleTag<?>, Integer> mapping) {
+      this.mapping = mapping;
+    }
+
+    @Override
+    public DoFnRunners.OutputManager create(final Output<StreamRecord<RawUnionValue>> output) {
+      return new DoFnRunners.OutputManager() {
+        @Override
+        public <T> void output(TupleTag<T> tag, WindowedValue<T> value) {
+          int intTag = mapping.get(tag);
+          output.collect(new StreamRecord<>(new RawUnionValue(intTag, value)));
+        }
+      };
+    }
+  }
+
+  /**
+   * {@link StepContext} for running {@link DoFn DoFns} on Flink. This does not allow
+   * accessing state or timer internals.
+   */
+  protected class StepContext implements ExecutionContext.StepContext {
+
+    @Override
+    public String getStepName() {
+      return null;
+    }
+
+    @Override
+    public String getTransformName() {
+      return null;
+    }
+
+    @Override
+    public void noteOutput(WindowedValue<?> output) {}
+
+    @Override
+    public void noteOutput(TupleTag<?> tag, WindowedValue<?> output) {}
+
+    @Override
+    public <T, W extends BoundedWindow> void writePCollectionViewData(
+        TupleTag<?> tag,
+        Iterable<WindowedValue<T>> data,
+        Coder<Iterable<WindowedValue<T>>> dataCoder,
+        W window,
+        Coder<W> windowCoder) throws IOException {
+      throw new UnsupportedOperationException("Writing side-input data is not supported.");
+    }
+
+    @Override
+    public StateInternals<?> stateInternals() {
+      return stateInternals;
+    }
+
+    @Override
+    public TimerInternals timerInternals() {
+      return timerInternals;
+    }
+  }
+
+  private class FlinkTimerInternals implements TimerInternals {
+
+    @Override
+    public void setTimer(
+        StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
+      setTimer(TimerData.of(timerId, namespace, target, timeDomain));
+    }
+
+    @Deprecated
+    @Override
+    public void setTimer(TimerData timerKey) {
+      long time = timerKey.getTimestamp().getMillis();
+      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
+        timerService.registerEventTimeTimer(timerKey, time);
+      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
+        timerService.registerProcessingTimeTimer(timerKey, time);
+      } else {
+        throw new UnsupportedOperationException(
+            "Unsupported time domain: " + timerKey.getDomain());
+      }
+    }
+
+    @Deprecated
+    @Override
+    public void deleteTimer(StateNamespace namespace, String timerId) {
+      throw new UnsupportedOperationException(
+          "Canceling of a timer by ID is not yet supported.");
+    }
+
+    @Override
+    public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
+      throw new UnsupportedOperationException(
+          "Canceling of a timer by ID is not yet supported.");
+    }
+
+    @Deprecated
+    @Override
+    public void deleteTimer(TimerData timerKey) {
+      long time = timerKey.getTimestamp().getMillis();
+      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
+        timerService.deleteEventTimeTimer(timerKey, time);
+      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
+        timerService.deleteProcessingTimeTimer(timerKey, time);
+      } else {
+        throw new UnsupportedOperationException(
+            "Unsupported time domain: " + timerKey.getDomain());
+      }
+    }
+
+    @Override
+    public Instant currentProcessingTime() {
+      return new Instant(timerService.currentProcessingTime());
+    }
+
+    @Nullable
+    @Override
+    public Instant currentSynchronizedProcessingTime() {
+      return new Instant(timerService.currentProcessingTime());
+    }
+
+    @Override
+    public Instant currentInputWatermarkTime() {
+      return new Instant(Math.min(currentInputWatermark, getPushbackWatermarkHold()));
+    }
+
+    @Nullable
+    @Override
+    public Instant currentOutputWatermarkTime() {
+      return new Instant(currentOutputWatermark);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java
new file mode 100644
index 0000000..dce2e68
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import java.nio.ByteBuffer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+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.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+/**
+ * {@link KeySelector} that retrieves a key from a {@link KV}. This will return
+ * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures
+ * that all key comparisons/hashing happen on the encoded form.
+ */
+public class KvToByteBufferKeySelector<K, V>
+    implements KeySelector<WindowedValue<KV<K, V>>, ByteBuffer>,
+    ResultTypeQueryable<ByteBuffer> {
+
+  private final Coder<K> keyCoder;
+
+  public KvToByteBufferKeySelector(Coder<K> keyCoder) {
+    this.keyCoder = keyCoder;
+  }
+
+  @Override
+  public ByteBuffer getKey(WindowedValue<KV<K, V>> value) throws Exception {
+    K key = value.getValue().getKey();
+    byte[] keyBytes = CoderUtils.encodeToByteArray(keyCoder, key);
+    return ByteBuffer.wrap(keyBytes);
+  }
+
+  @Override
+  public TypeInformation<ByteBuffer> getProducedType() {
+    return new GenericTypeInfo<>(ByteBuffer.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
new file mode 100644
index 0000000..e843660
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import java.util.Collections;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * Singleton keyed word item.
+ */
+public class SingletonKeyedWorkItem<K, ElemT> implements KeyedWorkItem<K, ElemT> {
+
+  final K key;
+  final WindowedValue<ElemT> value;
+
+  public SingletonKeyedWorkItem(K key, WindowedValue<ElemT> value) {
+    this.key = key;
+    this.value = value;
+  }
+
+  @Override
+  public K key() {
+    return key;
+  }
+
+  public WindowedValue<ElemT> value() {
+    return value;
+  }
+
+  @Override
+  public Iterable<TimerInternals.TimerData> timersIterable() {
+    return Collections.EMPTY_LIST;
+  }
+
+  @Override
+  public Iterable<WindowedValue<ElemT>> elementsIterable() {
+    return Collections.singletonList(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
new file mode 100644
index 0000000..9a52330
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItemCoder;
+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.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * Singleton keyed work item coder.
+ */
+public class SingletonKeyedWorkItemCoder<K, ElemT>
+    extends StandardCoder<SingletonKeyedWorkItem<K, ElemT>> {
+  /**
+   * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window
+   * coder.
+   */
+  public static <K, ElemT> SingletonKeyedWorkItemCoder<K, ElemT> of(
+      Coder<K> keyCoder, Coder<ElemT> elemCoder, Coder<? extends BoundedWindow> windowCoder) {
+    return new SingletonKeyedWorkItemCoder<>(keyCoder, elemCoder, windowCoder);
+  }
+
+  @JsonCreator
+  public static <K, ElemT> SingletonKeyedWorkItemCoder<K, ElemT> of(
+      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
+    checkArgument(components.size() == 3, "Expecting 3 components, got %s", components.size());
+    @SuppressWarnings("unchecked")
+    Coder<K> keyCoder = (Coder<K>) components.get(0);
+    @SuppressWarnings("unchecked")
+    Coder<ElemT> elemCoder = (Coder<ElemT>) components.get(1);
+    @SuppressWarnings("unchecked")
+    Coder<? extends BoundedWindow> windowCoder = (Coder<? extends BoundedWindow>) components.get(2);
+    return new SingletonKeyedWorkItemCoder<>(keyCoder, elemCoder, windowCoder);
+  }
+
+  private final Coder<K> keyCoder;
+  private final Coder<ElemT> elemCoder;
+  private final Coder<? extends BoundedWindow> windowCoder;
+  private final WindowedValue.FullWindowedValueCoder<ElemT> valueCoder;
+
+  private SingletonKeyedWorkItemCoder(
+      Coder<K> keyCoder, Coder<ElemT> elemCoder, Coder<? extends BoundedWindow> windowCoder) {
+    this.keyCoder = keyCoder;
+    this.elemCoder = elemCoder;
+    this.windowCoder = windowCoder;
+    valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder);
+  }
+
+  public Coder<K> getKeyCoder() {
+    return keyCoder;
+  }
+
+  public Coder<ElemT> getElementCoder() {
+    return elemCoder;
+  }
+
+  @Override
+  public void encode(SingletonKeyedWorkItem<K, ElemT> value,
+                     OutputStream outStream,
+                     Context context)
+      throws CoderException, IOException {
+    keyCoder.encode(value.key(), outStream, context.nested());
+    valueCoder.encode(value.value, outStream, context);
+  }
+
+  @Override
+  public SingletonKeyedWorkItem<K, ElemT> decode(InputStream inStream, Context context)
+      throws CoderException, IOException {
+    K key = keyCoder.decode(inStream, context.nested());
+    WindowedValue<ElemT> value = valueCoder.decode(inStream, context);
+    return new SingletonKeyedWorkItem<>(key, value);
+  }
+
+  @Override
+  public List<? extends Coder<?>> getCoderArguments() {
+    return ImmutableList.of(keyCoder, elemCoder, windowCoder);
+  }
+
+  @Override
+  public void verifyDeterministic() throws NonDeterministicException {
+    keyCoder.verifyDeterministic();
+    elemCoder.verifyDeterministic();
+    windowCoder.verifyDeterministic();
+  }
+
+  /**
+   * {@inheritDoc}.
+   *
+   * {@link KeyedWorkItemCoder} is not consistent with equals as it can return a
+   * {@link KeyedWorkItem} of a type different from the originally encoded type.
+   */
+  @Override
+  public boolean consistentWithEquals() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
new file mode 100644
index 0000000..40f70e4
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import org.apache.beam.runners.core.ElementAndRestriction;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItems;
+import org.apache.beam.runners.core.OutputAndTimeBoundedSplittableProcessElementInvoker;
+import org.apache.beam.runners.core.OutputWindowedValue;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateInternalsFactory;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.core.TimerInternalsFactory;
+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.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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.TupleTag;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Flink operator for executing splittable {@link DoFn DoFns}. Specifically, for executing
+ * the {@code @ProcessElement} method of a splittable {@link DoFn}.
+ */
+public class SplittableDoFnOperator<
+    InputT, FnOutputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
+    extends DoFnOperator<
+    KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, FnOutputT, OutputT> {
+
+  public SplittableDoFnOperator(
+      DoFn<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, FnOutputT> doFn,
+      Coder<
+          WindowedValue<
+              KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>> inputCoder,
+      TupleTag<FnOutputT> mainOutputTag,
+      List<TupleTag<?>> additionalOutputTags,
+      OutputManagerFactory<OutputT> outputManagerFactory,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Map<Integer, PCollectionView<?>> sideInputTagMapping,
+      Collection<PCollectionView<?>> sideInputs,
+      PipelineOptions options,
+      Coder<?> keyCoder) {
+    super(
+        doFn,
+        inputCoder,
+        mainOutputTag,
+        additionalOutputTags,
+        outputManagerFactory,
+        windowingStrategy,
+        sideInputTagMapping,
+        sideInputs,
+        options,
+        keyCoder);
+
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+
+    checkState(doFn instanceof SplittableParDo.ProcessFn);
+
+    StateInternalsFactory<String> stateInternalsFactory = new StateInternalsFactory<String>() {
+      @Override
+      public StateInternals<String> stateInternalsForKey(String key) {
+        //this will implicitly be keyed by the key of the incoming
+        // element or by the key of a firing timer
+        return (StateInternals<String>) stateInternals;
+      }
+    };
+    TimerInternalsFactory<String> timerInternalsFactory = new TimerInternalsFactory<String>() {
+      @Override
+      public TimerInternals timerInternalsForKey(String key) {
+        //this will implicitly be keyed like the StateInternalsFactory
+        return timerInternals;
+      }
+    };
+
+    ((SplittableParDo.ProcessFn) doFn).setStateInternalsFactory(stateInternalsFactory);
+    ((SplittableParDo.ProcessFn) doFn).setTimerInternalsFactory(timerInternalsFactory);
+    ((SplittableParDo.ProcessFn) doFn).setProcessElementInvoker(
+        new OutputAndTimeBoundedSplittableProcessElementInvoker<>(
+            doFn,
+            serializedOptions.getPipelineOptions(),
+            new OutputWindowedValue<FnOutputT>() {
+              @Override
+              public void outputWindowedValue(
+                  FnOutputT output,
+                  Instant timestamp,
+                  Collection<? extends BoundedWindow> windows,
+                  PaneInfo pane) {
+                outputManager.output(
+                    mainOutputTag,
+                    WindowedValue.of(output, timestamp, windows, pane));
+              }
+
+              @Override
+              public <AdditionalOutputT> void outputWindowedValue(
+                  TupleTag<AdditionalOutputT> tag,
+                  AdditionalOutputT output,
+                  Instant timestamp,
+                  Collection<? extends BoundedWindow> windows,
+                  PaneInfo pane) {
+                outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane));
+              }
+            },
+            sideInputReader,
+            Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()),
+            10000,
+            Duration.standardSeconds(10)));
+  }
+
+  @Override
+  public void fireTimer(InternalTimer<?, TimerInternals.TimerData> timer) {
+    doFnRunner.processElement(WindowedValue.valueInGlobalWindow(
+        KeyedWorkItems.<String, ElementAndRestriction<InputT, RestrictionT>>timersWorkItem(
+            (String) stateInternals.getKey(),
+            Collections.singletonList(timer.getNamespace()))));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
new file mode 100644
index 0000000..9b2136c
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import static org.apache.beam.runners.core.TimerInternals.TimerData;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItems;
+import org.apache.beam.runners.core.StateInternals;
+import org.apache.beam.runners.core.StateInternalsFactory;
+import org.apache.beam.runners.core.SystemReduceFn;
+import org.apache.beam.runners.core.TimerInternals;
+import org.apache.beam.runners.core.TimerInternalsFactory;
+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.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+
+/**
+ * Flink operator for executing window {@link DoFn DoFns}.
+ */
+public class WindowDoFnOperator<K, InputT, OutputT>
+    extends DoFnOperator<KeyedWorkItem<K, InputT>, KV<K, OutputT>, WindowedValue<KV<K, OutputT>>> {
+
+  private final SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> systemReduceFn;
+
+  public WindowDoFnOperator(
+      SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> systemReduceFn,
+      Coder<WindowedValue<KeyedWorkItem<K, InputT>>> inputCoder,
+      TupleTag<KV<K, OutputT>> mainOutputTag,
+      List<TupleTag<?>> additionalOutputTags,
+      OutputManagerFactory<WindowedValue<KV<K, OutputT>>> outputManagerFactory,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Map<Integer, PCollectionView<?>> sideInputTagMapping,
+      Collection<PCollectionView<?>> sideInputs,
+      PipelineOptions options,
+      Coder<K> keyCoder) {
+    super(
+        null,
+        inputCoder,
+        mainOutputTag,
+        additionalOutputTags,
+        outputManagerFactory,
+        windowingStrategy,
+        sideInputTagMapping,
+        sideInputs,
+        options,
+        keyCoder);
+
+    this.systemReduceFn = systemReduceFn;
+
+  }
+
+  @Override
+  protected DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> getDoFn() {
+    StateInternalsFactory<K> stateInternalsFactory = new StateInternalsFactory<K>() {
+      @Override
+      public StateInternals<K> stateInternalsForKey(K key) {
+        //this will implicitly be keyed by the key of the incoming
+        // element or by the key of a firing timer
+        return (StateInternals<K>) stateInternals;
+      }
+    };
+    TimerInternalsFactory<K> timerInternalsFactory = new TimerInternalsFactory<K>() {
+      @Override
+      public TimerInternals timerInternalsForKey(K key) {
+        //this will implicitly be keyed like the StateInternalsFactory
+        return timerInternals;
+      }
+    };
+
+    // we have to do the unchecked cast because GroupAlsoByWindowViaWindowSetDoFn.create
+    // has the window type as generic parameter while WindowingStrategy is almost always
+    // untyped.
+    @SuppressWarnings("unchecked")
+    DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFn =
+        GroupAlsoByWindowViaWindowSetNewDoFn.create(
+            windowingStrategy, stateInternalsFactory, timerInternalsFactory, sideInputReader,
+                (SystemReduceFn) systemReduceFn, outputManager, mainOutputTag);
+    return doFn;
+  }
+
+  @Override
+  public void fireTimer(InternalTimer<?, TimerData> timer) {
+    doFnRunner.processElement(WindowedValue.valueInGlobalWindow(
+        KeyedWorkItems.<K, InputT>timersWorkItem(
+            (K) stateInternals.getKey(),
+            Collections.singletonList(timer.getNamespace()))));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
new file mode 100644
index 0000000..1dff367
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
+
+import java.nio.ByteBuffer;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+/**
+ * {@link KeySelector} that retrieves a key from a {@link KeyedWorkItem}. This will return
+ * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures
+ * that all key comparisons/hashing happen on the encoded form.
+ */
+public class WorkItemKeySelector<K, V>
+    implements KeySelector<WindowedValue<SingletonKeyedWorkItem<K, V>>, ByteBuffer>,
+    ResultTypeQueryable<ByteBuffer> {
+
+  private final Coder<K> keyCoder;
+
+  public WorkItemKeySelector(Coder<K> keyCoder) {
+    this.keyCoder = keyCoder;
+  }
+
+  @Override
+  public ByteBuffer getKey(WindowedValue<SingletonKeyedWorkItem<K, V>> value) throws Exception {
+    K key = value.getValue().key();
+    byte[] keyBytes = CoderUtils.encodeToByteArray(keyCoder, key);
+    return ByteBuffer.wrap(keyBytes);
+  }
+
+  @Override
+  public TypeInformation<ByteBuffer> getProducedType() {
+    return new GenericTypeInfo<>(ByteBuffer.class);
+  }
+}


[17/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
[BEAM-1994] Remove Flink examples package


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

Branch: refs/heads/master
Commit: cdd2544ba6dd6ac4aa80c65ecd8e01ab3cf664aa
Parents: 8a00f22
Author: Isma�l Mej�a <ie...@apache.org>
Authored: Tue Apr 18 17:31:07 2017 +0200
Committer: Isma�l Mej�a <ie...@apache.org>
Committed: Wed Apr 19 13:37:06 2017 +0200

----------------------------------------------------------------------
 ...PostCommit_Java_ValidatesRunner_Flink.groovy |    2 +-
 runners/flink/examples/pom.xml                  |  130 ---
 .../beam/runners/flink/examples/TFIDF.java      |  455 --------
 .../beam/runners/flink/examples/WordCount.java  |  129 ---
 .../runners/flink/examples/package-info.java    |   22 -
 .../flink/examples/streaming/AutoComplete.java  |  400 -------
 .../flink/examples/streaming/JoinExamples.java  |  154 ---
 .../examples/streaming/WindowedWordCount.java   |  141 ---
 .../flink/examples/streaming/package-info.java  |   22 -
 runners/flink/pom.xml                           |  275 ++++-
 runners/flink/runner/pom.xml                    |  330 ------
 .../flink/DefaultParallelismFactory.java        |   39 -
 .../flink/FlinkBatchPipelineTranslator.java     |  139 ---
 .../flink/FlinkBatchTransformTranslators.java   |  723 ------------
 .../flink/FlinkBatchTranslationContext.java     |  153 ---
 .../flink/FlinkDetachedRunnerResult.java        |   75 --
 .../FlinkPipelineExecutionEnvironment.java      |  241 ----
 .../runners/flink/FlinkPipelineOptions.java     |  101 --
 .../runners/flink/FlinkPipelineTranslator.java  |   53 -
 .../apache/beam/runners/flink/FlinkRunner.java  |  232 ----
 .../runners/flink/FlinkRunnerRegistrar.java     |   62 --
 .../beam/runners/flink/FlinkRunnerResult.java   |   98 --
 .../flink/FlinkStreamingPipelineTranslator.java |  276 -----
 .../FlinkStreamingTransformTranslators.java     | 1044 -----------------
 .../flink/FlinkStreamingTranslationContext.java |  130 ---
 .../flink/FlinkStreamingViewOverrides.java      |  372 -------
 .../flink/PipelineTranslationOptimizer.java     |   72 --
 .../beam/runners/flink/TestFlinkRunner.java     |   84 --
 .../beam/runners/flink/TranslationMode.java     |   31 -
 .../apache/beam/runners/flink/package-info.java |   22 -
 .../functions/FlinkAggregatorFactory.java       |   53 -
 .../functions/FlinkAssignContext.java           |   63 --
 .../functions/FlinkAssignWindows.java           |   49 -
 .../functions/FlinkDoFnFunction.java            |  161 ---
 .../FlinkMergingNonShuffleReduceFunction.java   |  228 ----
 .../FlinkMergingPartialReduceFunction.java      |  201 ----
 .../functions/FlinkMergingReduceFunction.java   |  199 ----
 .../FlinkMultiOutputPruningFunction.java        |   50 -
 .../functions/FlinkNoOpStepContext.java         |   73 --
 .../functions/FlinkPartialReduceFunction.java   |  172 ---
 .../functions/FlinkReduceFunction.java          |  173 ---
 .../functions/FlinkSideInputReader.java         |   80 --
 .../functions/FlinkStatefulDoFnFunction.java    |  198 ----
 .../functions/SideInputInitializer.java         |   73 --
 .../translation/functions/package-info.java     |   22 -
 .../runners/flink/translation/package-info.java |   22 -
 .../translation/types/CoderTypeInformation.java |  120 --
 .../translation/types/CoderTypeSerializer.java  |  132 ---
 .../types/EncodedValueComparator.java           |  195 ----
 .../types/EncodedValueSerializer.java           |  113 --
 .../types/EncodedValueTypeInformation.java      |   98 --
 .../types/InspectableByteArrayOutputStream.java |   34 -
 .../flink/translation/types/KvKeySelector.java  |   50 -
 .../flink/translation/types/package-info.java   |   22 -
 .../utils/SerializedPipelineOptions.java        |   67 --
 .../flink/translation/utils/package-info.java   |   22 -
 .../wrappers/DataInputViewWrapper.java          |   58 -
 .../wrappers/DataOutputViewWrapper.java         |   51 -
 .../SerializableFnAggregatorWrapper.java        |   98 --
 .../translation/wrappers/SourceInputFormat.java |  150 ---
 .../translation/wrappers/SourceInputSplit.java  |   52 -
 .../translation/wrappers/package-info.java      |   22 -
 .../wrappers/streaming/DoFnOperator.java        |  774 -------------
 .../streaming/KvToByteBufferKeySelector.java    |   56 -
 .../streaming/SingletonKeyedWorkItem.java       |   56 -
 .../streaming/SingletonKeyedWorkItemCoder.java  |  126 ---
 .../streaming/SplittableDoFnOperator.java       |  150 ---
 .../wrappers/streaming/WindowDoFnOperator.java  |  117 --
 .../wrappers/streaming/WorkItemKeySelector.java |   56 -
 .../streaming/io/BoundedSourceWrapper.java      |  218 ----
 .../streaming/io/UnboundedSocketSource.java     |  249 -----
 .../streaming/io/UnboundedSourceWrapper.java    |  476 --------
 .../wrappers/streaming/io/package-info.java     |   22 -
 .../wrappers/streaming/package-info.java        |   22 -
 .../state/FlinkBroadcastStateInternals.java     |  865 --------------
 .../state/FlinkKeyGroupStateInternals.java      |  487 --------
 .../state/FlinkSplitStateInternals.java         |  260 -----
 .../streaming/state/FlinkStateInternals.java    | 1053 ------------------
 .../state/KeyGroupCheckpointedOperator.java     |   35 -
 .../state/KeyGroupRestoringOperator.java        |   32 -
 .../wrappers/streaming/state/package-info.java  |   22 -
 .../runner/src/main/resources/log4j.properties  |   23 -
 .../flink/EncodedValueComparatorTest.java       |   70 --
 .../runners/flink/FlinkRunnerRegistrarTest.java |   48 -
 .../beam/runners/flink/FlinkTestPipeline.java   |   72 --
 .../beam/runners/flink/PipelineOptionsTest.java |  184 ---
 .../beam/runners/flink/ReadSourceITCase.java    |   85 --
 .../flink/ReadSourceStreamingITCase.java        |   74 --
 .../beam/runners/flink/WriteSinkITCase.java     |  192 ----
 .../flink/streaming/DoFnOperatorTest.java       |  600 ----------
 .../FlinkBroadcastStateInternalsTest.java       |  245 ----
 .../FlinkKeyGroupStateInternalsTest.java        |  262 -----
 .../streaming/FlinkSplitStateInternalsTest.java |  101 --
 .../streaming/FlinkStateInternalsTest.java      |  395 -------
 .../flink/streaming/GroupByNullKeyTest.java     |  124 ---
 .../flink/streaming/TestCountingSource.java     |  254 -----
 .../streaming/TopWikipediaSessionsITCase.java   |  133 ---
 .../streaming/UnboundedSourceWrapperTest.java   |  464 --------
 .../runners/flink/streaming/package-info.java   |   22 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../flink/DefaultParallelismFactory.java        |   39 +
 .../flink/FlinkBatchPipelineTranslator.java     |  139 +++
 .../flink/FlinkBatchTransformTranslators.java   |  723 ++++++++++++
 .../flink/FlinkBatchTranslationContext.java     |  153 +++
 .../flink/FlinkDetachedRunnerResult.java        |   75 ++
 .../FlinkPipelineExecutionEnvironment.java      |  241 ++++
 .../runners/flink/FlinkPipelineOptions.java     |  101 ++
 .../runners/flink/FlinkPipelineTranslator.java  |   53 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  232 ++++
 .../runners/flink/FlinkRunnerRegistrar.java     |   62 ++
 .../beam/runners/flink/FlinkRunnerResult.java   |   98 ++
 .../flink/FlinkStreamingPipelineTranslator.java |  276 +++++
 .../FlinkStreamingTransformTranslators.java     | 1044 +++++++++++++++++
 .../flink/FlinkStreamingTranslationContext.java |  130 +++
 .../flink/FlinkStreamingViewOverrides.java      |  372 +++++++
 .../flink/PipelineTranslationOptimizer.java     |   72 ++
 .../beam/runners/flink/TestFlinkRunner.java     |   84 ++
 .../beam/runners/flink/TranslationMode.java     |   31 +
 .../apache/beam/runners/flink/package-info.java |   22 +
 .../functions/FlinkAggregatorFactory.java       |   53 +
 .../functions/FlinkAssignContext.java           |   63 ++
 .../functions/FlinkAssignWindows.java           |   49 +
 .../functions/FlinkDoFnFunction.java            |  161 +++
 .../FlinkMergingNonShuffleReduceFunction.java   |  228 ++++
 .../FlinkMergingPartialReduceFunction.java      |  201 ++++
 .../functions/FlinkMergingReduceFunction.java   |  199 ++++
 .../FlinkMultiOutputPruningFunction.java        |   50 +
 .../functions/FlinkNoOpStepContext.java         |   73 ++
 .../functions/FlinkPartialReduceFunction.java   |  172 +++
 .../functions/FlinkReduceFunction.java          |  173 +++
 .../functions/FlinkSideInputReader.java         |   80 ++
 .../functions/FlinkStatefulDoFnFunction.java    |  198 ++++
 .../functions/SideInputInitializer.java         |   73 ++
 .../translation/functions/package-info.java     |   22 +
 .../runners/flink/translation/package-info.java |   22 +
 .../translation/types/CoderTypeInformation.java |  120 ++
 .../translation/types/CoderTypeSerializer.java  |  132 +++
 .../types/EncodedValueComparator.java           |  195 ++++
 .../types/EncodedValueSerializer.java           |  113 ++
 .../types/EncodedValueTypeInformation.java      |   98 ++
 .../types/InspectableByteArrayOutputStream.java |   34 +
 .../flink/translation/types/KvKeySelector.java  |   50 +
 .../flink/translation/types/package-info.java   |   22 +
 .../utils/SerializedPipelineOptions.java        |   67 ++
 .../flink/translation/utils/package-info.java   |   22 +
 .../wrappers/DataInputViewWrapper.java          |   58 +
 .../wrappers/DataOutputViewWrapper.java         |   51 +
 .../SerializableFnAggregatorWrapper.java        |   98 ++
 .../translation/wrappers/SourceInputFormat.java |  150 +++
 .../translation/wrappers/SourceInputSplit.java  |   52 +
 .../translation/wrappers/package-info.java      |   22 +
 .../wrappers/streaming/DoFnOperator.java        |  774 +++++++++++++
 .../streaming/KvToByteBufferKeySelector.java    |   56 +
 .../streaming/SingletonKeyedWorkItem.java       |   56 +
 .../streaming/SingletonKeyedWorkItemCoder.java  |  126 +++
 .../streaming/SplittableDoFnOperator.java       |  150 +++
 .../wrappers/streaming/WindowDoFnOperator.java  |  117 ++
 .../wrappers/streaming/WorkItemKeySelector.java |   56 +
 .../streaming/io/BoundedSourceWrapper.java      |  218 ++++
 .../streaming/io/UnboundedSocketSource.java     |  249 +++++
 .../streaming/io/UnboundedSourceWrapper.java    |  476 ++++++++
 .../wrappers/streaming/io/package-info.java     |   22 +
 .../wrappers/streaming/package-info.java        |   22 +
 .../state/FlinkBroadcastStateInternals.java     |  865 ++++++++++++++
 .../state/FlinkKeyGroupStateInternals.java      |  487 ++++++++
 .../state/FlinkSplitStateInternals.java         |  260 +++++
 .../streaming/state/FlinkStateInternals.java    | 1053 ++++++++++++++++++
 .../state/KeyGroupCheckpointedOperator.java     |   35 +
 .../state/KeyGroupRestoringOperator.java        |   32 +
 .../wrappers/streaming/state/package-info.java  |   22 +
 .../flink/src/main/resources/log4j.properties   |   23 +
 .../flink/EncodedValueComparatorTest.java       |   70 ++
 .../runners/flink/FlinkRunnerRegistrarTest.java |   48 +
 .../beam/runners/flink/FlinkTestPipeline.java   |   72 ++
 .../beam/runners/flink/PipelineOptionsTest.java |  184 +++
 .../beam/runners/flink/ReadSourceITCase.java    |   85 ++
 .../flink/ReadSourceStreamingITCase.java        |   74 ++
 .../beam/runners/flink/WriteSinkITCase.java     |  192 ++++
 .../flink/streaming/DoFnOperatorTest.java       |  600 ++++++++++
 .../FlinkBroadcastStateInternalsTest.java       |  245 ++++
 .../FlinkKeyGroupStateInternalsTest.java        |  262 +++++
 .../streaming/FlinkSplitStateInternalsTest.java |  101 ++
 .../streaming/FlinkStateInternalsTest.java      |  395 +++++++
 .../flink/streaming/GroupByNullKeyTest.java     |  124 +++
 .../flink/streaming/TestCountingSource.java     |  254 +++++
 .../streaming/TopWikipediaSessionsITCase.java   |  133 +++
 .../streaming/UnboundedSourceWrapperTest.java   |  464 ++++++++
 .../runners/flink/streaming/package-info.java   |   22 +
 .../src/test/resources/log4j-test.properties    |   27 +
 189 files changed, 15765 insertions(+), 17293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Flink.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Flink.groovy b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Flink.groovy
index 411106d..5b228bc 100644
--- a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Flink.groovy
+++ b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Flink.groovy
@@ -39,5 +39,5 @@ mavenJob('beam_PostCommit_Java_ValidatesRunner_Flink') {
     'Run Flink ValidatesRunner')
 
   // Maven goals for this job.
-  goals('-B -e clean verify -am -pl runners/flink/runner -Plocal-validates-runner-tests -Pvalidates-runner-tests')
+  goals('-B -e clean verify -am -pl runners/flink -Plocal-validates-runner-tests -Pvalidates-runner-tests')
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
deleted file mode 100644
index aaf76d9..0000000
--- a/runners/flink/examples/pom.xml
+++ /dev/null
@@ -1,130 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-    Licensed to the Apache Software Foundation (ASF) under one or more
-    contributor license agreements.  See the NOTICE file distributed with
-    this work for additional information regarding copyright ownership.
-    The ASF licenses this file to You under the Apache License, Version 2.0
-    (the "License"); you may not use this file except in compliance with
-    the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software
-    distributed under the License is distributed on an "AS IS" BASIS,
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and
-    limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-runners-flink_2.10-examples</artifactId>
-
-  <name>Apache Beam :: Runners :: Flink :: Examples</name>
-
-  <packaging>jar</packaging>
-
-  <properties>
-    <!-- Default parameters for mvn exec:java -->
-    <flink.examples.input>kinglear.txt</flink.examples.input>
-    <flink.examples.output>wordcounts.txt</flink.examples.output>
-    <flink.examples.parallelism>-1</flink.examples.parallelism>
-  </properties>
-
-  <profiles>
-    <profile>
-      <id>disable-validates-runner-tests</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>validates-runner-tests</id>
-                <configuration>
-                  <skip>true</skip>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-flink_2.10</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-connector-kafka-0.8_2.10</artifactId>
-      <version>${flink.version}</version>
-    </dependency>
-
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <!-- disable for now until dependencies are cleaned up -->
-              <failOnWarning>false</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <configuration>
-          <executable>java</executable>
-          <arguments>
-            <argument>--runner=org.apache.beam.runners.flink.FlinkRunner</argument>
-            <argument>--parallelism=${flink.examples.parallelism}</argument>
-            <argument>--input=${flink.examples.input}</argument>
-            <argument>--output=${flink.examples.output}</argument>
-          </arguments>
-        </configuration>
-      </plugin>
-
-    </plugins>
-
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 8e1df08..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
+++ /dev/null
@@ -1,455 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.examples;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkRunner;
-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.Distinct;
-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.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.PBegin;
-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.TupleTag;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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=BlockingDataflowRunner
- * 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 interface Options extends PipelineOptions, FlinkPipelineOptions {
-    @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);
-      String[] directoryListing = directory.list();
-      if (directoryListing == null) {
-        throw new IOException(
-            "Directory " + absoluteUri + " is not a valid path or IO Error occurred.");
-      }
-      for (String entry : directoryListing) {
-        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<PBegin, PCollection<KV<URI, String>>> {
-    private static final long serialVersionUID = 0;
-
-    // transient because PTransform is not really meant to be serialized.
-    // see note on PTransform
-    private final transient 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>> expand(PBegin 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(" + uriString + ")", TextIO.Read.from(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>>>> {
-    private static final long serialVersionUID = 0;
-
-    public ComputeTfIdf() { }
-
-    @Override
-    public PCollection<KV<String, KV<URI, Double>>> expand(
-        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("DistinctDocs", Distinct.<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("SplitWords", ParDo.of(new DoFn<KV<URI, String>, KV<URI, String>>() {
-            private static final long serialVersionUID = 0;
-
-            @ProcessElement
-            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 \u201clove\u201d 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("DistinctWords", Distinct.<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("ShiftKeys", ParDo.of(
-              new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
-                private static final long serialVersionUID = 0;
-
-                @ProcessElement
-                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<>();
-      final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<>();
-      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("ComputeTermFrequencies", ParDo.of(
-              new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
-                private static final long serialVersionUID = 0;
-
-                @ProcessElement
-                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("ComputeDocFrequencies", ParDo
-              .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
-                private static final long serialVersionUID = 0;
-
-                @ProcessElement
-                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));
-                }
-              }).withSideInputs(totalDocuments));
-
-      // Join the term frequency and document frequency
-      // collections, each keyed on the word.
-      final TupleTag<KV<URI, Double>> tfTag = new TupleTag<>();
-      final TupleTag<Double> dfTag = new TupleTag<>();
-      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.
-
-      return wordToUriAndTfAndDf
-          .apply("ComputeTfIdf", ParDo.of(
-              new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
-                private static final long serialVersionUID = 0;
-
-                @ProcessElement
-                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)));
-                  }
-                }
-              }));
-    }
-
-    // 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 static final long serialVersionUID = 0;
-
-    private String output;
-
-    public WriteTfIdf(String output) {
-      this.output = output;
-    }
-
-    @Override
-    public PDone expand(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
-      return wordToUriAndTfIdf
-          .apply("Format", ParDo.of(new DoFn<KV<String, KV<URI, Double>>, String>() {
-            private static final long serialVersionUID = 0;
-
-            @ProcessElement
-            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);
-
-    options.setRunner(FlinkRunner.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/beam/blob/cdd2544b/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
deleted file mode 100644
index 6ae4cf8..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.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 org.apache.beam.runners.flink.examples;
-
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.sdk.Pipeline;
-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.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.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Wordcount pipeline.
- */
-public class WordCount {
-
-  /**
-   * Function to extract words.
-   */
-  public static class ExtractWordsFn extends DoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", Sum.ofLongs());
-
-    @ProcessElement
-    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);
-        }
-      }
-    }
-  }
-
-  /**
-   * PTransform counting words.
-   */
-  public static class CountWords extends PTransform<PCollection<String>,
-                    PCollection<KV<String, Long>>> {
-    @Override
-    public PCollection<KV<String, Long>> expand(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;
-    }
-  }
-
-  /** 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();
-    }
-  }
-
-  /**
-   * Options supported by {@link WordCount}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  public interface Options extends PipelineOptions, FlinkPipelineOptions {
-    @Description("Path of the file to read from")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Path of the file to write to")
-    @Validation.Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args) {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
-        .as(Options.class);
-    options.setRunner(FlinkRunner.class);
-
-    Pipeline p = Pipeline.create(options);
-
-    p.apply("ReadLines", TextIO.Read.from(options.getInput()))
-        .apply(new CountWords())
-        .apply(MapElements.via(new FormatAsTextFn()))
-        .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
-
-    p.run();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
deleted file mode 100644
index b0ecb56..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Flink Beam runner exemple.
- */
-package org.apache.beam.runners.flink.examples;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index d07df29..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
+++ /dev/null
@@ -1,400 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.examples.streaming;
-
-import java.io.IOException;
-import java.util.List;
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-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.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.Aggregator;
-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.Sum;
-import org.apache.beam.sdk.transforms.Top;
-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.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.apache.beam.sdk.values.PCollectionList;
-import org.joda.time.Duration;
-
-/**
- * To run the example, first open a socket on a terminal by executing the command:
- * <ul>
- *   <li><code>nc -lk 9999</code>
- * </ul>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-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 static final long serialVersionUID = 0;
-
-    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>>> expand(PCollection<String> input) {
-      PCollection<CompletionCandidate> candidates = input
-        // First count how often each token appears.
-        .apply(Count.<String>perElement())
-
-        // Map the KV outputs of Count into our own CompletionCandiate class.
-        .apply("CreateCompletionCandidates", ParDo.of(
-            new DoFn<KV<String, Long>, CompletionCandidate>() {
-              private static final long serialVersionUID = 0;
-
-              @ProcessElement
-              public void processElement(ProcessContext c) {
-                CompletionCandidate cand = new CompletionCandidate(c.element().getKey(),
-                    c.element().getValue());
-                c.output(cand);
-              }
-            }));
-
-      // 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 static final long serialVersionUID = 0;
-
-    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>>> expand(
-        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> {
-      private static final long serialVersionUID = 0;
-
-      @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 static final long serialVersionUID = 0;
-
-    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>>> {
-      private static final long serialVersionUID = 0;
-
-      @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> {
-      private static final long serialVersionUID = 0;
-
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        for (CompletionCandidate cc : c.element().getValue()) {
-          c.output(cc);
-        }
-      }
-    }
-
-    @Override
-    public PCollectionList<KV<String, List<CompletionCandidate>>> expand(
-          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.by(new SerializableFunction<CompletionCandidate, Boolean>() {
-              private static final long serialVersionUID = 0;
-
-              @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 static final long serialVersionUID = 0;
-
-    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;
-    }
-    @ProcessElement
-      public void processElement(ProcessContext c) {
-      String word = c.element().value;
-      for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
-        KV<String, CompletionCandidate> kv = KV.of(word.substring(0, i), c.element());
-        c.output(kv);
-      }
-    }
-  }
-
-  /**
-   * 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 String getValue() {
-      return value;
-    }
-
-    // Empty constructor required for Avro decoding.
-    @SuppressWarnings("unused")
-    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 + "]";
-    }
-  }
-
-  static class ExtractWordsFn extends DoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-            createAggregator("emptyLines", Sum.ofLongs());
-
-    @ProcessElement
-    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);
-        }
-      }
-    }
-  }
-
-  /**
-   * Takes as input a the top candidates per prefix, and emits an entity suitable for writing to
-   * Datastore.
-   */
-  static class FormatForPerTaskLocalFile
-      extends DoFn<KV<String, List<CompletionCandidate>>, String> {
-
-    private static final long serialVersionUID = 0;
-
-    @ProcessElement
-    public void processElement(ProcessContext c, BoundedWindow window) {
-      StringBuilder str = new StringBuilder();
-      KV<String, List<CompletionCandidate>> elem = c.element();
-
-      str.append(elem.getKey() + " @ " + window + " -> ");
-      for (CompletionCandidate cand: elem.getValue()) {
-        str.append(cand.toString() + " ");
-      }
-      System.out.println(str.toString());
-      c.output(str.toString());
-    }
-  }
-
-  /**
-   * Options supported by this class.
-   *
-   * <p>Inherits standard Dataflow configuration options.
-   */
-  private interface Options extends WindowedWordCount.StreamingWordCountOptions {
-    @Description("Whether to use the recursive algorithm")
-    @Default.Boolean(true)
-    Boolean getRecursive();
-    void setRecursive(Boolean value);
-  }
-
-  public static void main(String[] args) throws IOException {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    options.setStreaming(true);
-    options.setCheckpointingInterval(1000L);
-    options.setNumberOfExecutionRetries(5);
-    options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkRunner.class);
-
-
-    WindowFn<Object, ?> windowFn =
-        FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
-
-    // Create the pipeline.
-    Pipeline p = Pipeline.create(options);
-    PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
-      .apply("WordStream", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
-      .apply(ParDo.of(new ExtractWordsFn()))
-      .apply(Window.<String>into(windowFn)
-              .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
-            .discardingFiredPanes())
-      .apply(ComputeTopCompletions.top(10, options.getRecursive()));
-
-    toWrite
-      .apply("FormatForPerTaskFile", ParDo.of(new FormatForPerTaskLocalFile()))
-      .apply(TextIO.Write.to("./outputAutoComplete.txt"));
-
-    p.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 8fefc9f..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.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 org.apache.beam.runners.flink.examples.streaming;
-
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-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.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.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Duration;
-
-/**
- * To run the example, first open two sockets on two terminals by executing the commands:
- * <ul>
- *   <li><code>nc -lk 9999</code>, and
- *   <li><code>nc -lk 9998</code>
- * </ul>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-public class JoinExamples {
-
-  static PCollection<String> joinEvents(PCollection<String> streamA,
-                      PCollection<String> streamB) throws Exception {
-
-    final TupleTag<String> firstInfoTag = new TupleTag<>();
-    final TupleTag<String> secondInfoTag = new TupleTag<>();
-
-    // transform both input collections to tuple collections, where the keys are country
-    // codes in both cases.
-    PCollection<KV<String, String>> firstInfo = streamA.apply(
-        ParDo.of(new ExtractEventDataFn()));
-    PCollection<KV<String, String>> secondInfo = streamB.apply(
-        ParDo.of(new ExtractEventDataFn()));
-
-    // country code 'key' -> CGBKR (<event info>, <country name>)
-    PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
-        .of(firstInfoTag, firstInfo)
-        .and(secondInfoTag, secondInfo)
-        .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("Process", ParDo.of(
-            new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
-              private static final long serialVersionUID = 0;
-
-              @ProcessElement
-              public void processElement(ProcessContext c) {
-                KV<String, CoGbkResult> e = c.element();
-                String key = e.getKey();
-
-                String defaultA = "NO_VALUE";
-
-                // the following getOnly is a bit tricky because it expects to have
-                // EXACTLY ONE value in the corresponding stream and for the corresponding key.
-
-                String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
-                for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
-                  // Generate a string that combines information from both collection values
-                  c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
-                }
-              }
-            }));
-
-    return finalResultCollection
-        .apply("Format", ParDo.of(new DoFn<KV<String, String>, String>() {
-          private static final long serialVersionUID = 0;
-
-          @ProcessElement
-          public void processElement(ProcessContext c) {
-            String result = c.element().getKey() + " -> " + c.element().getValue();
-            System.out.println(result);
-            c.output(result);
-          }
-        }));
-  }
-
-  static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
-    private static final long serialVersionUID = 0;
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      String line = c.element().toLowerCase();
-      String key = line.split("\\s")[0];
-      c.output(KV.of(key, line));
-    }
-  }
-
-  private interface Options extends WindowedWordCount.StreamingWordCountOptions {
-
-  }
-
-  public static void main(String[] args) throws Exception {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    options.setStreaming(true);
-    options.setCheckpointingInterval(1000L);
-    options.setNumberOfExecutionRetries(5);
-    options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkRunner.class);
-
-    WindowFn<Object, ?> windowFn = FixedWindows.of(
-        Duration.standardSeconds(options.getWindowSize()));
-
-    Pipeline p = Pipeline.create(options);
-
-    // the following two 'applys' create multiple inputs to our pipeline, one for each
-    // of our two input sources.
-    PCollection<String> streamA = p
-        .apply("FirstStream", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
-        .apply(Window.<String>into(windowFn)
-            .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
-            .discardingFiredPanes());
-    PCollection<String> streamB = p
-        .apply("SecondStream", Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)))
-        .apply(Window.<String>into(windowFn)
-            .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
-            .discardingFiredPanes());
-
-    PCollection<String> formattedResults = joinEvents(streamA, streamB);
-    formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
-    p.run();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 792c214..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.examples.streaming;
-
-import java.io.IOException;
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-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.Aggregator;
-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.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-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.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * To run the example, first open a socket on a terminal by executing the command:
- * <ul>
- *   <li><code>nc -lk 9999</code>
- * </ul>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-public class WindowedWordCount {
-
-  private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
-
-  static final long WINDOW_SIZE = 10;  // Default window duration in seconds
-  static final long SLIDE_SIZE = 5;  // Default window slide in seconds
-
-  static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      String row = c.element().getKey() + " - " + c.element().getValue() + " @ "
-          + c.timestamp().toString();
-      c.output(row);
-    }
-  }
-
-  static class ExtractWordsFn extends DoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", Sum.ofLongs());
-
-    @ProcessElement
-    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);
-        }
-      }
-    }
-  }
-
-  /**
-   * Pipeline options.
-   */
-  public interface StreamingWordCountOptions
-      extends org.apache.beam.runners.flink.examples.WordCount.Options {
-    @Description("Sliding window duration, in seconds")
-    @Default.Long(WINDOW_SIZE)
-    Long getWindowSize();
-
-    void setWindowSize(Long value);
-
-    @Description("Window slide, in seconds")
-    @Default.Long(SLIDE_SIZE)
-    Long getSlide();
-
-    void setSlide(Long value);
-  }
-
-  public static void main(String[] args) throws IOException {
-    StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-        .as(StreamingWordCountOptions.class);
-    options.setStreaming(true);
-    options.setWindowSize(10L);
-    options.setSlide(5L);
-    options.setCheckpointingInterval(1000L);
-    options.setNumberOfExecutionRetries(5);
-    options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkRunner.class);
-
-    LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize()
-        + " sec. and a slide of " + options.getSlide());
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    PCollection<String> words = pipeline
-        .apply("StreamingWordCount",
-            Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
-        .apply(ParDo.of(new ExtractWordsFn()))
-        .apply(Window.<String>into(SlidingWindows.of(
-            Duration.standardSeconds(options.getWindowSize()))
-            .every(Duration.standardSeconds(options.getSlide())))
-            .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
-            .discardingFiredPanes());
-
-    PCollection<KV<String, Long>> wordCounts =
-        words.apply(Count.<String>perElement());
-
-    wordCounts.apply(ParDo.of(new FormatAsStringFn()))
-        .apply(TextIO.Write.to("./outputWordCount.txt"));
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
deleted file mode 100644
index 58f41b6..0000000
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Flink Beam runner exemple.
- */
-package org.apache.beam.runners.flink.examples.streaming;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index a5c5ea0..351035e 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -26,22 +26,97 @@
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <artifactId>beam-runners-flink-parent</artifactId>
+  <artifactId>beam-runners-flink</artifactId>
   <name>Apache Beam :: Runners :: Flink</name>
-
-  <packaging>pom</packaging>
-
-  <modules>
-    <module>runner</module>
-    <module>examples</module>
-  </modules>
+  <packaging>jar</packaging>
 
   <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <flink.version>1.2.0</flink.version>
   </properties>
 
+  <profiles>
+    <profile>
+      <id>local-validates-runner-tests</id>
+      <activation><activeByDefault>false</activeByDefault></activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <executions>
+
+              <!-- This configures the inherited validates-runner-tests
+                   execution to execute with a local Flink instance. -->
+              <execution>
+                <id>validates-runner-tests</id>
+                <phase>integration-test</phase>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <groups>org.apache.beam.sdk.testing.ValidatesRunner</groups>
+                  <excludedGroups>
+                    org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders,
+                    org.apache.beam.sdk.testing.UsesSplittableParDo,
+                    org.apache.beam.sdk.testing.UsesAttemptedMetrics,
+                    org.apache.beam.sdk.testing.UsesCommittedMetrics,
+                    org.apache.beam.sdk.testing.UsesTestStream
+                  </excludedGroups>
+                  <parallel>none</parallel>
+                  <failIfNoTests>true</failIfNoTests>
+                  <dependenciesToScan>
+                    <dependency>org.apache.beam:beam-sdks-java-core</dependency>
+                  </dependenciesToScan>
+                  <systemPropertyVariables>
+                    <beamTestPipelineOptions>
+                      [
+                      "--runner=TestFlinkRunner",
+                      "--streaming=false"
+                      ]
+                    </beamTestPipelineOptions>
+                  </systemPropertyVariables>
+                </configuration>
+              </execution>
+
+              <!-- This second execution runs the tests in streaming mode -->
+              <execution>
+                <id>streaming-validates-runner-tests</id>
+                <phase>integration-test</phase>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <groups>org.apache.beam.sdk.testing.ValidatesRunner</groups>
+                  <excludedGroups>
+                    org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders,
+                    org.apache.beam.sdk.testing.UsesSetState,
+                    org.apache.beam.sdk.testing.UsesMapState,
+                    org.apache.beam.sdk.testing.UsesAttemptedMetrics,
+                    org.apache.beam.sdk.testing.UsesCommittedMetrics,
+                    org.apache.beam.sdk.testing.UsesTestStream
+                  </excludedGroups>
+                  <parallel>none</parallel>
+                  <failIfNoTests>true</failIfNoTests>
+                  <dependenciesToScan>
+                    <dependency>org.apache.beam:beam-sdks-java-core</dependency>
+                  </dependenciesToScan>
+                  <systemPropertyVariables>
+                    <beamTestPipelineOptions>
+                      [
+                      "--runner=TestFlinkRunner",
+                      "--streaming=true"
+                      ]
+                    </beamTestPipelineOptions>
+                  </systemPropertyVariables>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
   <build>
     <pluginManagement>
       <plugins>
@@ -89,19 +164,103 @@
     <!-- Flink dependencies -->
     <dependency>
       <groupId>org.apache.flink</groupId>
+      <artifactId>flink-clients_2.10</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-core</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-java</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-runtime_2.10</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
       <artifactId>flink-streaming-java_2.10</artifactId>
       <version>${flink.version}</version>
     </dependency>
 
+    <!-- For testing -->
     <dependency>
       <groupId>org.apache.flink</groupId>
       <artifactId>flink-core</artifactId>
       <version>${flink.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-runtime_2.10</artifactId>
+      <version>${flink.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
 
+    <!-- Beam -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-java</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-construction-java</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
     </dependency>
 
     <dependency>
@@ -113,5 +272,101 @@
       <groupId>joda-time</groupId>
       <artifactId>joda-time</artifactId>
     </dependency>
+
+    <!--
+    Force an upgrade on the version of Apache Commons from Flink to support DEFLATE compression.
+    -->
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-compress</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <!-- Test scoped -->
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Depend on test jar to scan for ValidatesRunner tests -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.10</artifactId>
+      <version>${flink.version}</version>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-test-utils_2.10</artifactId>
+      <version>${flink.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>apacheds-jdbm1</artifactId>
+          <groupId>org.apache.directory.jdbm</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Optional Pipeline Registration -->
+    <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <optional>true</optional>
+    </dependency>
+
+    <!-- transitive test dependencies from beam-sdk-java-core -->
+    <dependency>
+      <groupId>com.fasterxml.jackson.dataformat</groupId>
+      <artifactId>jackson-dataformat-yaml</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-common-fn-api</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>


[12/18] beam git commit: [BEAM-1994] Remove Flink examples package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
deleted file mode 100644
index 9a52330..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItemCoder;
-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.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * Singleton keyed work item coder.
- */
-public class SingletonKeyedWorkItemCoder<K, ElemT>
-    extends StandardCoder<SingletonKeyedWorkItem<K, ElemT>> {
-  /**
-   * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window
-   * coder.
-   */
-  public static <K, ElemT> SingletonKeyedWorkItemCoder<K, ElemT> of(
-      Coder<K> keyCoder, Coder<ElemT> elemCoder, Coder<? extends BoundedWindow> windowCoder) {
-    return new SingletonKeyedWorkItemCoder<>(keyCoder, elemCoder, windowCoder);
-  }
-
-  @JsonCreator
-  public static <K, ElemT> SingletonKeyedWorkItemCoder<K, ElemT> of(
-      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
-    checkArgument(components.size() == 3, "Expecting 3 components, got %s", components.size());
-    @SuppressWarnings("unchecked")
-    Coder<K> keyCoder = (Coder<K>) components.get(0);
-    @SuppressWarnings("unchecked")
-    Coder<ElemT> elemCoder = (Coder<ElemT>) components.get(1);
-    @SuppressWarnings("unchecked")
-    Coder<? extends BoundedWindow> windowCoder = (Coder<? extends BoundedWindow>) components.get(2);
-    return new SingletonKeyedWorkItemCoder<>(keyCoder, elemCoder, windowCoder);
-  }
-
-  private final Coder<K> keyCoder;
-  private final Coder<ElemT> elemCoder;
-  private final Coder<? extends BoundedWindow> windowCoder;
-  private final WindowedValue.FullWindowedValueCoder<ElemT> valueCoder;
-
-  private SingletonKeyedWorkItemCoder(
-      Coder<K> keyCoder, Coder<ElemT> elemCoder, Coder<? extends BoundedWindow> windowCoder) {
-    this.keyCoder = keyCoder;
-    this.elemCoder = elemCoder;
-    this.windowCoder = windowCoder;
-    valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder);
-  }
-
-  public Coder<K> getKeyCoder() {
-    return keyCoder;
-  }
-
-  public Coder<ElemT> getElementCoder() {
-    return elemCoder;
-  }
-
-  @Override
-  public void encode(SingletonKeyedWorkItem<K, ElemT> value,
-                     OutputStream outStream,
-                     Context context)
-      throws CoderException, IOException {
-    keyCoder.encode(value.key(), outStream, context.nested());
-    valueCoder.encode(value.value, outStream, context);
-  }
-
-  @Override
-  public SingletonKeyedWorkItem<K, ElemT> decode(InputStream inStream, Context context)
-      throws CoderException, IOException {
-    K key = keyCoder.decode(inStream, context.nested());
-    WindowedValue<ElemT> value = valueCoder.decode(inStream, context);
-    return new SingletonKeyedWorkItem<>(key, value);
-  }
-
-  @Override
-  public List<? extends Coder<?>> getCoderArguments() {
-    return ImmutableList.of(keyCoder, elemCoder, windowCoder);
-  }
-
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException {
-    keyCoder.verifyDeterministic();
-    elemCoder.verifyDeterministic();
-    windowCoder.verifyDeterministic();
-  }
-
-  /**
-   * {@inheritDoc}.
-   *
-   * {@link KeyedWorkItemCoder} is not consistent with equals as it can return a
-   * {@link KeyedWorkItem} of a type different from the originally encoded type.
-   */
-  @Override
-  public boolean consistentWithEquals() {
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
deleted file mode 100644
index 40f70e4..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import org.apache.beam.runners.core.ElementAndRestriction;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.OutputAndTimeBoundedSplittableProcessElementInvoker;
-import org.apache.beam.runners.core.OutputWindowedValue;
-import org.apache.beam.runners.core.SplittableParDo;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateInternalsFactory;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.TimerInternalsFactory;
-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.splittabledofn.RestrictionTracker;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-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.TupleTag;
-import org.apache.flink.streaming.api.operators.InternalTimer;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Flink operator for executing splittable {@link DoFn DoFns}. Specifically, for executing
- * the {@code @ProcessElement} method of a splittable {@link DoFn}.
- */
-public class SplittableDoFnOperator<
-    InputT, FnOutputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
-    extends DoFnOperator<
-    KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, FnOutputT, OutputT> {
-
-  public SplittableDoFnOperator(
-      DoFn<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, FnOutputT> doFn,
-      Coder<
-          WindowedValue<
-              KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>> inputCoder,
-      TupleTag<FnOutputT> mainOutputTag,
-      List<TupleTag<?>> additionalOutputTags,
-      OutputManagerFactory<OutputT> outputManagerFactory,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<Integer, PCollectionView<?>> sideInputTagMapping,
-      Collection<PCollectionView<?>> sideInputs,
-      PipelineOptions options,
-      Coder<?> keyCoder) {
-    super(
-        doFn,
-        inputCoder,
-        mainOutputTag,
-        additionalOutputTags,
-        outputManagerFactory,
-        windowingStrategy,
-        sideInputTagMapping,
-        sideInputs,
-        options,
-        keyCoder);
-
-  }
-
-  @Override
-  public void open() throws Exception {
-    super.open();
-
-    checkState(doFn instanceof SplittableParDo.ProcessFn);
-
-    StateInternalsFactory<String> stateInternalsFactory = new StateInternalsFactory<String>() {
-      @Override
-      public StateInternals<String> stateInternalsForKey(String key) {
-        //this will implicitly be keyed by the key of the incoming
-        // element or by the key of a firing timer
-        return (StateInternals<String>) stateInternals;
-      }
-    };
-    TimerInternalsFactory<String> timerInternalsFactory = new TimerInternalsFactory<String>() {
-      @Override
-      public TimerInternals timerInternalsForKey(String key) {
-        //this will implicitly be keyed like the StateInternalsFactory
-        return timerInternals;
-      }
-    };
-
-    ((SplittableParDo.ProcessFn) doFn).setStateInternalsFactory(stateInternalsFactory);
-    ((SplittableParDo.ProcessFn) doFn).setTimerInternalsFactory(timerInternalsFactory);
-    ((SplittableParDo.ProcessFn) doFn).setProcessElementInvoker(
-        new OutputAndTimeBoundedSplittableProcessElementInvoker<>(
-            doFn,
-            serializedOptions.getPipelineOptions(),
-            new OutputWindowedValue<FnOutputT>() {
-              @Override
-              public void outputWindowedValue(
-                  FnOutputT output,
-                  Instant timestamp,
-                  Collection<? extends BoundedWindow> windows,
-                  PaneInfo pane) {
-                outputManager.output(
-                    mainOutputTag,
-                    WindowedValue.of(output, timestamp, windows, pane));
-              }
-
-              @Override
-              public <AdditionalOutputT> void outputWindowedValue(
-                  TupleTag<AdditionalOutputT> tag,
-                  AdditionalOutputT output,
-                  Instant timestamp,
-                  Collection<? extends BoundedWindow> windows,
-                  PaneInfo pane) {
-                outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane));
-              }
-            },
-            sideInputReader,
-            Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()),
-            10000,
-            Duration.standardSeconds(10)));
-  }
-
-  @Override
-  public void fireTimer(InternalTimer<?, TimerInternals.TimerData> timer) {
-    doFnRunner.processElement(WindowedValue.valueInGlobalWindow(
-        KeyedWorkItems.<String, ElementAndRestriction<InputT, RestrictionT>>timersWorkItem(
-            (String) stateInternals.getKey(),
-            Collections.singletonList(timer.getNamespace()))));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
deleted file mode 100644
index 9b2136c..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import static org.apache.beam.runners.core.TimerInternals.TimerData;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateInternalsFactory;
-import org.apache.beam.runners.core.SystemReduceFn;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.TimerInternalsFactory;
-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.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.flink.streaming.api.operators.InternalTimer;
-
-/**
- * Flink operator for executing window {@link DoFn DoFns}.
- */
-public class WindowDoFnOperator<K, InputT, OutputT>
-    extends DoFnOperator<KeyedWorkItem<K, InputT>, KV<K, OutputT>, WindowedValue<KV<K, OutputT>>> {
-
-  private final SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> systemReduceFn;
-
-  public WindowDoFnOperator(
-      SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> systemReduceFn,
-      Coder<WindowedValue<KeyedWorkItem<K, InputT>>> inputCoder,
-      TupleTag<KV<K, OutputT>> mainOutputTag,
-      List<TupleTag<?>> additionalOutputTags,
-      OutputManagerFactory<WindowedValue<KV<K, OutputT>>> outputManagerFactory,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<Integer, PCollectionView<?>> sideInputTagMapping,
-      Collection<PCollectionView<?>> sideInputs,
-      PipelineOptions options,
-      Coder<K> keyCoder) {
-    super(
-        null,
-        inputCoder,
-        mainOutputTag,
-        additionalOutputTags,
-        outputManagerFactory,
-        windowingStrategy,
-        sideInputTagMapping,
-        sideInputs,
-        options,
-        keyCoder);
-
-    this.systemReduceFn = systemReduceFn;
-
-  }
-
-  @Override
-  protected DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> getDoFn() {
-    StateInternalsFactory<K> stateInternalsFactory = new StateInternalsFactory<K>() {
-      @Override
-      public StateInternals<K> stateInternalsForKey(K key) {
-        //this will implicitly be keyed by the key of the incoming
-        // element or by the key of a firing timer
-        return (StateInternals<K>) stateInternals;
-      }
-    };
-    TimerInternalsFactory<K> timerInternalsFactory = new TimerInternalsFactory<K>() {
-      @Override
-      public TimerInternals timerInternalsForKey(K key) {
-        //this will implicitly be keyed like the StateInternalsFactory
-        return timerInternals;
-      }
-    };
-
-    // we have to do the unchecked cast because GroupAlsoByWindowViaWindowSetDoFn.create
-    // has the window type as generic parameter while WindowingStrategy is almost always
-    // untyped.
-    @SuppressWarnings("unchecked")
-    DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFn =
-        GroupAlsoByWindowViaWindowSetNewDoFn.create(
-            windowingStrategy, stateInternalsFactory, timerInternalsFactory, sideInputReader,
-                (SystemReduceFn) systemReduceFn, outputManager, mainOutputTag);
-    return doFn;
-  }
-
-  @Override
-  public void fireTimer(InternalTimer<?, TimerData> timer) {
-    doFnRunner.processElement(WindowedValue.valueInGlobalWindow(
-        KeyedWorkItems.<K, InputT>timersWorkItem(
-            (K) stateInternals.getKey(),
-            Collections.singletonList(timer.getNamespace()))));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
deleted file mode 100644
index 1dff367..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.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 org.apache.beam.runners.flink.translation.wrappers.streaming;
-
-import java.nio.ByteBuffer;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-
-/**
- * {@link KeySelector} that retrieves a key from a {@link KeyedWorkItem}. This will return
- * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures
- * that all key comparisons/hashing happen on the encoded form.
- */
-public class WorkItemKeySelector<K, V>
-    implements KeySelector<WindowedValue<SingletonKeyedWorkItem<K, V>>, ByteBuffer>,
-    ResultTypeQueryable<ByteBuffer> {
-
-  private final Coder<K> keyCoder;
-
-  public WorkItemKeySelector(Coder<K> keyCoder) {
-    this.keyCoder = keyCoder;
-  }
-
-  @Override
-  public ByteBuffer getKey(WindowedValue<SingletonKeyedWorkItem<K, V>> value) throws Exception {
-    K key = value.getValue().key();
-    byte[] keyBytes = CoderUtils.encodeToByteArray(keyCoder, key);
-    return ByteBuffer.wrap(keyBytes);
-  }
-
-  @Override
-  public TypeInformation<ByteBuffer> getProducedType() {
-    return new GenericTypeInfo<>(ByteBuffer.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
deleted file mode 100644
index 2ed5024..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-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.StoppableFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Wrapper for executing {@link BoundedSource BoundedSources} as a Flink Source.
- */
-public class BoundedSourceWrapper<OutputT>
-    extends RichParallelSourceFunction<WindowedValue<OutputT>>
-    implements StoppableFunction {
-
-  private static final Logger LOG = LoggerFactory.getLogger(BoundedSourceWrapper.class);
-
-  /**
-   * Keep the options so that we can initialize the readers.
-   */
-  private final SerializedPipelineOptions serializedOptions;
-
-  /**
-   * The split sources. We split them in the constructor to ensure that all parallel
-   * sources are consistent about the split sources.
-   */
-  private List<? extends BoundedSource<OutputT>> splitSources;
-
-  /**
-   * Make it a field so that we can access it in {@link #close()}.
-   */
-  private transient List<BoundedSource.BoundedReader<OutputT>> readers;
-
-  /**
-   * Initialize here and not in run() to prevent races where we cancel a job before run() is
-   * ever called or run() is called after cancel().
-   */
-  private volatile boolean isRunning = true;
-
-  @SuppressWarnings("unchecked")
-  public BoundedSourceWrapper(
-      PipelineOptions pipelineOptions,
-      BoundedSource<OutputT> source,
-      int parallelism) throws Exception {
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-
-    long desiredBundleSize = source.getEstimatedSizeBytes(pipelineOptions) / parallelism;
-
-    // get the splits early. we assume that the generated splits are stable,
-    // this is necessary so that the mapping of state to source is correct
-    // when restoring
-    splitSources = source.split(desiredBundleSize, pipelineOptions);
-  }
-
-  @Override
-  public void run(SourceContext<WindowedValue<OutputT>> ctx) throws Exception {
-
-    // figure out which split sources we're responsible for
-    int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-    int numSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
-
-    List<BoundedSource<OutputT>> localSources = new ArrayList<>();
-
-    for (int i = 0; i < splitSources.size(); i++) {
-      if (i % numSubtasks == subtaskIndex) {
-        localSources.add(splitSources.get(i));
-      }
-    }
-
-    LOG.info("Bounded Flink Source {}/{} is reading from sources: {}",
-        subtaskIndex,
-        numSubtasks,
-        localSources);
-
-    readers = new ArrayList<>();
-    // initialize readers from scratch
-    for (BoundedSource<OutputT> source : localSources) {
-      readers.add(source.createReader(serializedOptions.getPipelineOptions()));
-    }
-
-   if (readers.size() == 1) {
-      // the easy case, we just read from one reader
-      BoundedSource.BoundedReader<OutputT> reader = readers.get(0);
-
-      boolean dataAvailable = reader.start();
-      if (dataAvailable) {
-        emitElement(ctx, reader);
-      }
-
-      while (isRunning) {
-        dataAvailable = reader.advance();
-
-        if (dataAvailable)  {
-          emitElement(ctx, reader);
-        } else {
-          break;
-        }
-      }
-    } else {
-      // a bit more complicated, we are responsible for several readers
-      // loop through them and sleep if none of them had any data
-
-      int currentReader = 0;
-
-      // start each reader and emit data if immediately available
-      for (BoundedSource.BoundedReader<OutputT> reader : readers) {
-        boolean dataAvailable = reader.start();
-        if (dataAvailable) {
-          emitElement(ctx, reader);
-        }
-      }
-
-      // a flag telling us whether any of the readers had data
-      // if no reader had data, sleep for bit
-      boolean hadData = false;
-      while (isRunning && !readers.isEmpty()) {
-        BoundedSource.BoundedReader<OutputT> reader = readers.get(currentReader);
-        boolean dataAvailable = reader.advance();
-
-        if (dataAvailable) {
-          emitElement(ctx, reader);
-          hadData = true;
-        } else {
-          readers.remove(currentReader);
-          currentReader--;
-          if (readers.isEmpty()) {
-            break;
-          }
-        }
-
-        currentReader = (currentReader + 1) % readers.size();
-        if (currentReader == 0 && !hadData) {
-          Thread.sleep(50);
-        } else if (currentReader == 0) {
-          hadData = false;
-        }
-      }
-
-    }
-
-    // emit final Long.MAX_VALUE watermark, just to be sure
-    ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
-  }
-
-  /**
-   * Emit the current element from the given Reader. The reader is guaranteed to have data.
-   */
-  private void emitElement(
-      SourceContext<WindowedValue<OutputT>> ctx,
-      BoundedSource.BoundedReader<OutputT> reader) {
-    // make sure that reader state update and element emission are atomic
-    // with respect to snapshots
-    synchronized (ctx.getCheckpointLock()) {
-
-      OutputT item = reader.getCurrent();
-      Instant timestamp = reader.getCurrentTimestamp();
-
-      WindowedValue<OutputT> windowedValue =
-          WindowedValue.of(item, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
-      ctx.collectWithTimestamp(windowedValue, timestamp.getMillis());
-    }
-  }
-
-  @Override
-  public void close() throws Exception {
-    super.close();
-    if (readers != null) {
-      for (BoundedSource.BoundedReader<OutputT> reader: readers) {
-        reader.close();
-      }
-    }
-  }
-
-  @Override
-  public void cancel() {
-    isRunning = false;
-  }
-
-  @Override
-  public void stop() {
-    this.isRunning = false;
-  }
-
-  /**
-   * Visible so that we can check this in tests. Must not be used for anything else.
-   */
-  @VisibleForTesting
-  public List<? extends BoundedSource<OutputT>> getSplitSources() {
-    return splitSources;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index 910a33f..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Collections;
-import java.util.List;
-import java.util.NoSuchElementException;
-import javax.annotation.Nullable;
-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;
-
-/**
- * An example unbounded Beam source that reads input from a socket.
- * This is used mainly for testing and debugging.
- * */
-public class UnboundedSocketSource<CheckpointMarkT extends UnboundedSource.CheckpointMark>
-    extends UnboundedSource<String, CheckpointMarkT> {
-
-  private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
-
-  private static final long serialVersionUID = 1L;
-
-  private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
-
-  private static final int CONNECTION_TIMEOUT_TIME = 0;
-
-  private final String hostname;
-  private final int port;
-  private final char delimiter;
-  private final long maxNumRetries;
-  private final long delayBetweenRetries;
-
-  public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
-    this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
-  }
-
-  public UnboundedSocketSource(String hostname,
-                               int port,
-                               char delimiter,
-                               long maxNumRetries,
-                               long delayBetweenRetries) {
-    this.hostname = hostname;
-    this.port = port;
-    this.delimiter = delimiter;
-    this.maxNumRetries = maxNumRetries;
-    this.delayBetweenRetries = delayBetweenRetries;
-  }
-
-  public String getHostname() {
-    return this.hostname;
-  }
-
-  public int getPort() {
-    return this.port;
-  }
-
-  public char getDelimiter() {
-    return this.delimiter;
-  }
-
-  public long getMaxNumRetries() {
-    return this.maxNumRetries;
-  }
-
-  public long getDelayBetweenRetries() {
-    return this.delayBetweenRetries;
-  }
-
-  @Override
-  public List<? extends UnboundedSource<String, CheckpointMarkT>> split(
-      int desiredNumSplits,
-      PipelineOptions options) throws Exception {
-    return Collections.<UnboundedSource<String, CheckpointMarkT>>singletonList(this);
-  }
-
-  @Override
-  public UnboundedReader<String> createReader(PipelineOptions options,
-                                              @Nullable CheckpointMarkT checkpointMark) {
-    return new UnboundedSocketReader(this);
-  }
-
-  @Nullable
-  @Override
-  public Coder getCheckpointMarkCoder() {
-    // Flink and Dataflow have different checkpointing mechanisms.
-    // In our case we do not need a coder.
-    return null;
-  }
-
-  @Override
-  public void validate() {
-    checkArgument(port > 0 && port < 65536, "port is out of range");
-    checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), "
-        + "or -1 (infinite retries)");
-    checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
-  }
-
-  @Override
-  public Coder getDefaultOutputCoder() {
-    return DEFAULT_SOCKET_CODER;
-  }
-
-  /**
-   * Unbounded socket reader.
-   */
-  public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
-
-    private final UnboundedSocketSource source;
-
-    private Socket socket;
-    private BufferedReader reader;
-
-    private boolean isRunning;
-
-    private String currentRecord;
-
-    public UnboundedSocketReader(UnboundedSocketSource source) {
-      this.source = source;
-    }
-
-    private void openConnection() throws IOException {
-      this.socket = new Socket();
-      this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()),
-          CONNECTION_TIMEOUT_TIME);
-      this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
-      this.isRunning = true;
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      int attempt = 0;
-      while (!isRunning) {
-        try {
-          openConnection();
-          LOG.info("Connected to server socket " + this.source.getHostname() + ':'
-              + this.source.getPort());
-
-          return advance();
-        } catch (IOException e) {
-          LOG.info("Lost connection to server socket " + this.source.getHostname() + ':'
-              + this.source.getPort() + ". Retrying in "
-              + this.source.getDelayBetweenRetries() + " msecs...");
-
-          if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
-            try {
-              Thread.sleep(this.source.getDelayBetweenRetries());
-            } catch (InterruptedException e1) {
-              e1.printStackTrace();
-            }
-          } else {
-            this.isRunning = false;
-            break;
-          }
-        }
-      }
-      LOG.error("Unable to connect to host " + this.source.getHostname()
-          + " : " + this.source.getPort());
-      return false;
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      final StringBuilder buffer = new StringBuilder();
-      int data;
-      while (isRunning && (data = reader.read()) != -1) {
-        // check if the string is complete
-        if (data != this.source.getDelimiter()) {
-          buffer.append((char) data);
-        } else {
-          if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
-            buffer.setLength(buffer.length() - 1);
-          }
-          this.currentRecord = buffer.toString();
-          buffer.setLength(0);
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public byte[] getCurrentRecordId() throws NoSuchElementException {
-      return new byte[0];
-    }
-
-    @Override
-    public String getCurrent() throws NoSuchElementException {
-      return this.currentRecord;
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() throws NoSuchElementException {
-      return Instant.now();
-    }
-
-    @Override
-    public void close() throws IOException {
-      this.reader.close();
-      this.socket.close();
-      this.isRunning = false;
-      LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":"
-          + this.source.getPort() + ".");
-    }
-
-    @Override
-    public Instant getWatermark() {
-      return Instant.now();
-    }
-
-    @Override
-    public CheckpointMark getCheckpointMark() {
-      return null;
-    }
-
-    @Override
-    public UnboundedSource<String, ?> getCurrentSource() {
-      return this.source;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/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
deleted file mode 100644
index bb9b58a..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ /dev/null
@@ -1,476 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-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.values.KV;
-import org.apache.beam.sdk.values.TypeDescriptor;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.StoppableFunction;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Wrapper for executing {@link UnboundedSource UnboundedSources} as a Flink Source.
- */
-public class UnboundedSourceWrapper<
-    OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
-    extends RichParallelSourceFunction<WindowedValue<OutputT>>
-    implements ProcessingTimeCallback, StoppableFunction,
-    CheckpointListener, CheckpointedFunction {
-
-  private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceWrapper.class);
-
-  /**
-   * Keep the options so that we can initialize the localReaders.
-   */
-  private final SerializedPipelineOptions serializedOptions;
-
-  /**
-   * For snapshot and restore.
-   */
-  private final KvCoder<
-      ? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> checkpointCoder;
-
-  /**
-   * The split sources. We split them in the constructor to ensure that all parallel
-   * sources are consistent about the split sources.
-   */
-  private final List<? extends UnboundedSource<OutputT, CheckpointMarkT>> splitSources;
-
-  /**
-   * The local split sources. Assigned at runtime when the wrapper is executed in parallel.
-   */
-  private transient List<UnboundedSource<OutputT, CheckpointMarkT>> localSplitSources;
-
-  /**
-   * The local split readers. Assigned at runtime when the wrapper is executed in parallel.
-   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for
-   * emitting watermarks.
-   */
-  private transient List<UnboundedSource.UnboundedReader<OutputT>> localReaders;
-
-  /**
-   * Flag to indicate whether the source is running.
-   * Initialize here and not in run() to prevent races where we cancel a job before run() is
-   * ever called or run() is called after cancel().
-   */
-  private volatile boolean isRunning = true;
-
-  /**
-   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for registering new
-   * triggers.
-   */
-  private transient StreamingRuntimeContext runtimeContext;
-
-  /**
-   * Make it a field so that we can access it in {@link #onProcessingTime(long)} for emitting
-   * watermarks.
-   */
-  private transient SourceContext<WindowedValue<OutputT>> context;
-
-  /**
-   * Pending checkpoints which have not been acknowledged yet.
-   */
-  private transient LinkedHashMap<Long, List<CheckpointMarkT>> pendingCheckpoints;
-  /**
-   * Keep a maximum of 32 checkpoints for {@code CheckpointMark.finalizeCheckpoint()}.
-   */
-  private static final int MAX_NUMBER_PENDING_CHECKPOINTS = 32;
-
-  private transient ListState<KV<? extends
-      UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT>> stateForCheckpoint;
-
-  /**
-   * false if checkpointCoder is null or no restore state by starting first.
-   */
-  private transient boolean isRestored = false;
-
-  @SuppressWarnings("unchecked")
-  public UnboundedSourceWrapper(
-      PipelineOptions pipelineOptions,
-      UnboundedSource<OutputT, CheckpointMarkT> source,
-      int parallelism) throws Exception {
-    this.serializedOptions = new SerializedPipelineOptions(pipelineOptions);
-
-    if (source.requiresDeduping()) {
-      LOG.warn("Source {} requires deduping but Flink runner doesn't support this yet.", source);
-    }
-
-    Coder<CheckpointMarkT> checkpointMarkCoder = source.getCheckpointMarkCoder();
-    if (checkpointMarkCoder == null) {
-      LOG.info("No CheckpointMarkCoder specified for this source. Won't create snapshots.");
-      checkpointCoder = null;
-    } else {
-
-      Coder<? extends UnboundedSource<OutputT, CheckpointMarkT>> sourceCoder =
-          (Coder) SerializableCoder.of(new TypeDescriptor<UnboundedSource>() {
-          });
-
-      checkpointCoder = KvCoder.of(sourceCoder, checkpointMarkCoder);
-    }
-
-    // get the splits early. we assume that the generated splits are stable,
-    // this is necessary so that the mapping of state to source is correct
-    // when restoring
-    splitSources = source.split(parallelism, pipelineOptions);
-  }
-
-
-  /**
-   * Initialize and restore state before starting execution of the source.
-   */
-  @Override
-  public void open(Configuration parameters) throws Exception {
-    runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
-
-    // figure out which split sources we're responsible for
-    int subtaskIndex = runtimeContext.getIndexOfThisSubtask();
-    int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
-
-    localSplitSources = new ArrayList<>();
-    localReaders = new ArrayList<>();
-
-    pendingCheckpoints = new LinkedHashMap<>();
-
-    if (isRestored) {
-      // restore the splitSources from the checkpoint to ensure consistent ordering
-      for (KV<? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> restored:
-          stateForCheckpoint.get()) {
-        localSplitSources.add(restored.getKey());
-        localReaders.add(restored.getKey().createReader(
-            serializedOptions.getPipelineOptions(), restored.getValue()));
-      }
-    } else {
-      // initialize localReaders and localSources from scratch
-      for (int i = 0; i < splitSources.size(); i++) {
-        if (i % numSubtasks == subtaskIndex) {
-          UnboundedSource<OutputT, CheckpointMarkT> source =
-              splitSources.get(i);
-          UnboundedSource.UnboundedReader<OutputT> reader =
-              source.createReader(serializedOptions.getPipelineOptions(), null);
-          localSplitSources.add(source);
-          localReaders.add(reader);
-        }
-      }
-    }
-
-    LOG.info("Unbounded Flink Source {}/{} is reading from sources: {}",
-        subtaskIndex,
-        numSubtasks,
-        localSplitSources);
-  }
-
-  @Override
-  public void run(SourceContext<WindowedValue<OutputT>> ctx) throws Exception {
-
-    context = ctx;
-
-    if (localReaders.size() == 0) {
-      // do nothing, but still look busy ...
-      // also, output a Long.MAX_VALUE watermark since we know that we're not
-      // going to emit anything
-      // we can't return here since Flink requires that all operators stay up,
-      // otherwise checkpointing would not work correctly anymore
-      ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
-
-      // wait until this is canceled
-      final Object waitLock = new Object();
-      while (isRunning) {
-        try {
-          // Flink will interrupt us at some point
-          //noinspection SynchronizationOnLocalVariableOrMethodParameter
-          synchronized (waitLock) {
-            // don't wait indefinitely, in case something goes horribly wrong
-            waitLock.wait(1000);
-          }
-        } catch (InterruptedException e) {
-          if (!isRunning) {
-            // restore the interrupted state, and fall through the loop
-            Thread.currentThread().interrupt();
-          }
-        }
-      }
-    } else if (localReaders.size() == 1) {
-      // the easy case, we just read from one reader
-      UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(0);
-
-      boolean dataAvailable = reader.start();
-      if (dataAvailable) {
-        emitElement(ctx, reader);
-      }
-
-      setNextWatermarkTimer(this.runtimeContext);
-
-      while (isRunning) {
-        dataAvailable = reader.advance();
-
-        if (dataAvailable)  {
-          emitElement(ctx, reader);
-        } else {
-          Thread.sleep(50);
-        }
-      }
-    } else {
-      // a bit more complicated, we are responsible for several localReaders
-      // loop through them and sleep if none of them had any data
-
-      int numReaders = localReaders.size();
-      int currentReader = 0;
-
-      // start each reader and emit data if immediately available
-      for (UnboundedSource.UnboundedReader<OutputT> reader : localReaders) {
-        boolean dataAvailable = reader.start();
-        if (dataAvailable) {
-          emitElement(ctx, reader);
-        }
-      }
-
-      // a flag telling us whether any of the localReaders had data
-      // if no reader had data, sleep for bit
-      boolean hadData = false;
-      while (isRunning) {
-        UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(currentReader);
-        boolean dataAvailable = reader.advance();
-
-        if (dataAvailable) {
-          emitElement(ctx, reader);
-          hadData = true;
-        }
-
-        currentReader = (currentReader + 1) % numReaders;
-        if (currentReader == 0 && !hadData) {
-          Thread.sleep(50);
-        } else if (currentReader == 0) {
-          hadData = false;
-        }
-      }
-
-    }
-  }
-
-  /**
-   * Emit the current element from the given Reader. The reader is guaranteed to have data.
-   */
-  private void emitElement(
-      SourceContext<WindowedValue<OutputT>> ctx,
-      UnboundedSource.UnboundedReader<OutputT> reader) {
-    // make sure that reader state update and element emission are atomic
-    // with respect to snapshots
-    synchronized (ctx.getCheckpointLock()) {
-
-      OutputT item = reader.getCurrent();
-      Instant timestamp = reader.getCurrentTimestamp();
-
-      WindowedValue<OutputT> windowedValue =
-          WindowedValue.of(item, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
-      ctx.collectWithTimestamp(windowedValue, timestamp.getMillis());
-    }
-  }
-
-  @Override
-  public void close() throws Exception {
-    super.close();
-    if (localReaders != null) {
-      for (UnboundedSource.UnboundedReader<OutputT> reader: localReaders) {
-        reader.close();
-      }
-    }
-  }
-
-  @Override
-  public void cancel() {
-    isRunning = false;
-  }
-
-  @Override
-  public void stop() {
-    isRunning = false;
-  }
-
-  // ------------------------------------------------------------------------
-  //  Checkpoint and restore
-  // ------------------------------------------------------------------------
-
-  @Override
-  public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
-    if (!isRunning) {
-      LOG.debug("snapshotState() called on closed source");
-    } else {
-
-      if (checkpointCoder == null) {
-        // no checkpoint coder available in this source
-        return;
-      }
-
-      stateForCheckpoint.clear();
-
-      long checkpointId = functionSnapshotContext.getCheckpointId();
-
-      // we checkpoint the sources along with the CheckpointMarkT to ensure
-      // than we have a correct mapping of checkpoints to sources when
-      // restoring
-      List<CheckpointMarkT> checkpointMarks = new ArrayList<>(localSplitSources.size());
-
-      for (int i = 0; i < localSplitSources.size(); i++) {
-        UnboundedSource<OutputT, CheckpointMarkT> source = localSplitSources.get(i);
-        UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(i);
-
-        @SuppressWarnings("unchecked")
-        CheckpointMarkT mark = (CheckpointMarkT) reader.getCheckpointMark();
-        checkpointMarks.add(mark);
-        KV<UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT> kv =
-            KV.of(source, mark);
-        stateForCheckpoint.add(kv);
-      }
-
-      // cleanup old pending checkpoints and add new checkpoint
-      int diff = pendingCheckpoints.size() - MAX_NUMBER_PENDING_CHECKPOINTS;
-      if (diff >= 0) {
-        for (Iterator<Long> iterator = pendingCheckpoints.keySet().iterator();
-             diff >= 0;
-             diff--) {
-          iterator.next();
-          iterator.remove();
-        }
-      }
-      pendingCheckpoints.put(checkpointId, checkpointMarks);
-
-    }
-  }
-
-  @Override
-  public void initializeState(FunctionInitializationContext context) throws Exception {
-    if (checkpointCoder == null) {
-      // no checkpoint coder available in this source
-      return;
-    }
-
-    OperatorStateStore stateStore = context.getOperatorStateStore();
-    CoderTypeInformation<
-        KV<? extends UnboundedSource<OutputT, CheckpointMarkT>, CheckpointMarkT>>
-        typeInformation = (CoderTypeInformation) new CoderTypeInformation<>(checkpointCoder);
-    stateForCheckpoint = stateStore.getOperatorState(
-        new ListStateDescriptor<>(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME,
-            typeInformation.createSerializer(new ExecutionConfig())));
-
-    if (context.isRestored()) {
-      isRestored = true;
-      LOG.info("Having restore state in the UnbounedSourceWrapper.");
-    } else {
-      LOG.info("No restore state for UnbounedSourceWrapper.");
-    }
-  }
-
-  @Override
-  public void onProcessingTime(long timestamp) throws Exception {
-    if (this.isRunning) {
-      synchronized (context.getCheckpointLock()) {
-        // find minimum watermark over all localReaders
-        long watermarkMillis = Long.MAX_VALUE;
-        for (UnboundedSource.UnboundedReader<OutputT> reader: localReaders) {
-          Instant watermark = reader.getWatermark();
-          if (watermark != null) {
-            watermarkMillis = Math.min(watermark.getMillis(), watermarkMillis);
-          }
-        }
-        context.emitWatermark(new Watermark(watermarkMillis));
-      }
-      setNextWatermarkTimer(this.runtimeContext);
-    }
-  }
-
-  private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
-    if (this.isRunning) {
-      long watermarkInterval =  runtime.getExecutionConfig().getAutoWatermarkInterval();
-      long timeToNextWatermark = getTimeToNextWatermark(watermarkInterval);
-      runtime.getProcessingTimeService().registerTimer(timeToNextWatermark, this);
-    }
-  }
-
-  private long getTimeToNextWatermark(long watermarkInterval) {
-    return System.currentTimeMillis() + watermarkInterval;
-  }
-
-  /**
-   * Visible so that we can check this in tests. Must not be used for anything else.
-   */
-  @VisibleForTesting
-  public List<? extends UnboundedSource<OutputT, CheckpointMarkT>> getSplitSources() {
-    return splitSources;
-  }
-
-  /**
-   * Visible so that we can check this in tests. Must not be used for anything else.
-   */
-  @VisibleForTesting
-  public List<? extends UnboundedSource<OutputT, CheckpointMarkT>> getLocalSplitSources() {
-    return localSplitSources;
-  }
-
-  @Override
-  public void notifyCheckpointComplete(long checkpointId) throws Exception {
-
-    List<CheckpointMarkT> checkpointMarks = pendingCheckpoints.get(checkpointId);
-
-    if (checkpointMarks != null) {
-
-      // remove old checkpoints including the current one
-      Iterator<Long> iterator = pendingCheckpoints.keySet().iterator();
-      long currentId;
-      do {
-        currentId = iterator.next();
-        iterator.remove();
-      } while (currentId != checkpointId);
-
-      // confirm all marks
-      for (CheckpointMarkT mark : checkpointMarks) {
-        mark.finalizeCheckpoint();
-      }
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
deleted file mode 100644
index b431ce7..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming.io;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
deleted file mode 100644
index 0674871..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink.translation.wrappers.streaming;

http://git-wip-us.apache.org/repos/asf/beam/blob/cdd2544b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
deleted file mode 100644
index 3203446..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.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 org.apache.beam.runners.flink.translation.wrappers.streaming.state;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.MapCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-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.CombineContextFactory;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MapState;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.SetState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
-import org.apache.flink.runtime.state.OperatorStateBackend;
-
-/**
- * {@link StateInternals} that uses a Flink {@link DefaultOperatorStateBackend}
- * to manage the broadcast state.
- * The state is the same on all parallel instances of the operator.
- * So we just need store state of operator-0 in OperatorStateBackend.
- *
- * <p>Note: Ignore index of key.
- * Mainly for SideInputs.
- */
-public class FlinkBroadcastStateInternals<K> implements StateInternals<K> {
-
-  private int indexInSubtaskGroup;
-  private final DefaultOperatorStateBackend stateBackend;
-  // stateName -> <namespace, state>
-  private Map<String, Map<String, ?>> stateForNonZeroOperator;
-
-  public FlinkBroadcastStateInternals(int indexInSubtaskGroup, OperatorStateBackend stateBackend) {
-    //TODO flink do not yet expose through public API
-    this.stateBackend = (DefaultOperatorStateBackend) stateBackend;
-    this.indexInSubtaskGroup = indexInSubtaskGroup;
-    if (indexInSubtaskGroup != 0) {
-      stateForNonZeroOperator = new HashMap<>();
-    }
-  }
-
-  @Override
-  public K getKey() {
-    return null;
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address) {
-
-    return state(namespace, address, StateContexts.nullContext());
-  }
-
-  @Override
-  public <T extends State> T state(
-      final StateNamespace namespace,
-      StateTag<? super K, T> address,
-      final StateContext<?> context) {
-
-    return address.bind(new StateTag.StateBinder<K>() {
-
-      @Override
-      public <T> ValueState<T> bindValue(
-          StateTag<? super K, ValueState<T>> address,
-          Coder<T> coder) {
-
-        return new FlinkBroadcastValueState<>(stateBackend, address, namespace, coder);
-      }
-
-      @Override
-      public <T> BagState<T> bindBag(
-          StateTag<? super K, BagState<T>> address,
-          Coder<T> elemCoder) {
-
-        return new FlinkBroadcastBagState<>(stateBackend, address, namespace, elemCoder);
-      }
-
-      @Override
-      public <T> SetState<T> bindSet(
-          StateTag<? super K, SetState<T>> address,
-          Coder<T> elemCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", SetState.class.getSimpleName()));
-      }
-
-      @Override
-      public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
-          StateTag<? super K, MapState<KeyT, ValueT>> spec,
-          Coder<KeyT> mapKeyCoder, Coder<ValueT> mapValueCoder) {
-        throw new UnsupportedOperationException(
-            String.format("%s is not supported", MapState.class.getSimpleName()));
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT>
-      bindCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
-
-        return new FlinkCombiningState<>(
-            stateBackend, address, combineFn, namespace, accumCoder);
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
-        return new FlinkKeyedCombiningState<>(
-            stateBackend,
-            address,
-            combineFn,
-            namespace,
-            accumCoder,
-            FlinkBroadcastStateInternals.this);
-      }
-
-      @Override
-      public <InputT, AccumT, OutputT>
-      CombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
-          StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-          Coder<AccumT> accumCoder,
-          CombineWithContext.KeyedCombineFnWithContext<
-              ? super K, InputT, AccumT, OutputT> combineFn) {
-        return new FlinkCombiningStateWithContext<>(
-            stateBackend,
-            address,
-            combineFn,
-            namespace,
-            accumCoder,
-            FlinkBroadcastStateInternals.this,
-            CombineContextFactory.createFromStateContext(context));
-      }
-
-      @Override
-      public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(
-          StateTag<? super K, WatermarkHoldState<W>> address,
-          OutputTimeFn<? super W> outputTimeFn) {
-         throw new UnsupportedOperationException(
-             String.format("%s is not supported", WatermarkHoldState.class.getSimpleName()));
-      }
-    });
-  }
-
-  /**
-   * 1. The way we would use it is to only checkpoint anything from the operator
-   * with subtask index 0 because we assume that the state is the same on all
-   * parallel instances of the operator.
-   *
-   * <p>2. Use map to support namespace.
-   */
-  private abstract class AbstractBroadcastState<T> {
-
-    private String name;
-    private final StateNamespace namespace;
-    private final ListStateDescriptor<Map<String, T>> flinkStateDescriptor;
-    private final DefaultOperatorStateBackend flinkStateBackend;
-
-    AbstractBroadcastState(
-        DefaultOperatorStateBackend flinkStateBackend,
-        String name,
-        StateNamespace namespace,
-        Coder<T> coder) {
-      this.name = name;
-
-      this.namespace = namespace;
-      this.flinkStateBackend = flinkStateBackend;
-
-      CoderTypeInformation<Map<String, T>> typeInfo =
-          new CoderTypeInformation<>(MapCoder.of(StringUtf8Coder.of(), coder));
-
-      flinkStateDescriptor = new ListStateDescriptor<>(name,
-          typeInfo.createSerializer(new ExecutionConfig()));
-    }
-
-    /**
-     * Get map(namespce->T) from index 0.
-     */
-    Map<String, T> getMap() throws Exception {
-      if (indexInSubtaskGroup == 0) {
-        return getMapFromBroadcastState();
-      } else {
-        Map<String, T> result = (Map<String, T>) stateForNonZeroOperator.get(name);
-        // maybe restore from BroadcastState of Operator-0
-        if (result == null) {
-          result = getMapFromBroadcastState();
-          if (result != null) {
-            stateForNonZeroOperator.put(name, result);
-            // we don't need it anymore, must clear it.
-            flinkStateBackend.getBroadcastOperatorState(
-                flinkStateDescriptor).clear();
-          }
-        }
-        return result;
-      }
-    }
-
-    Map<String, T> getMapFromBroadcastState() throws Exception {
-      ListState<Map<String, T>> state = flinkStateBackend.getBroadcastOperatorState(
-          flinkStateDescriptor);
-      Iterable<Map<String, T>> iterable = state.get();
-      Map<String, T> ret = null;
-      if (iterable != null) {
-        // just use index 0
-        Iterator<Map<String, T>> iterator = iterable.iterator();
-        if (iterator.hasNext()) {
-          ret = iterator.next();
-        }
-      }
-      return ret;
-    }
-
-    /**
-     * Update map(namespce->T) from index 0.
-     */
-    void updateMap(Map<String, T> map) throws Exception {
-      if (indexInSubtaskGroup == 0) {
-        ListState<Map<String, T>> state = flinkStateBackend.getBroadcastOperatorState(
-            flinkStateDescriptor);
-        state.clear();
-        if (map.size() > 0) {
-          state.add(map);
-        }
-      } else {
-        if (map.size() == 0) {
-          stateForNonZeroOperator.remove(name);
-          // updateMap is always behind getMap,
-          // getMap will clear map in BroadcastOperatorState,
-          // we don't need clear here.
-        } else {
-          stateForNonZeroOperator.put(name, map);
-        }
-      }
-    }
-
-    void writeInternal(T input) {
-      try {
-        Map<String, T> map = getMap();
-        if (map == null) {
-          map = new HashMap<>();
-        }
-        map.put(namespace.stringKey(), input);
-        updateMap(map);
-      } catch (Exception e) {
-        throw new RuntimeException("Error updating state.", e);
-      }
-    }
-
-    T readInternal() {
-      try {
-        Map<String, T> map = getMap();
-        if (map == null) {
-          return null;
-        } else {
-          return map.get(namespace.stringKey());
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    void clearInternal() {
-      try {
-        Map<String, T> map = getMap();
-        if (map != null) {
-          map.remove(namespace.stringKey());
-          updateMap(map);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error clearing state.", e);
-      }
-    }
-
-  }
-
-  private class FlinkBroadcastValueState<K, T>
-      extends AbstractBroadcastState<T> implements ValueState<T> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, ValueState<T>> address;
-
-    FlinkBroadcastValueState(
-        DefaultOperatorStateBackend flinkStateBackend,
-        StateTag<? super K, ValueState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-      super(flinkStateBackend, address.getId(), namespace, coder);
-
-      this.namespace = namespace;
-      this.address = address;
-
-    }
-
-    @Override
-    public void write(T input) {
-      writeInternal(input);
-    }
-
-    @Override
-    public ValueState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public T read() {
-      return readInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkBroadcastValueState<?, ?> that = (FlinkBroadcastValueState<?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-  }
-
-  private class FlinkBroadcastBagState<K, T> extends AbstractBroadcastState<List<T>>
-      implements BagState<T> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, BagState<T>> address;
-
-    FlinkBroadcastBagState(
-        DefaultOperatorStateBackend flinkStateBackend,
-        StateTag<? super K, BagState<T>> address,
-        StateNamespace namespace,
-        Coder<T> coder) {
-      super(flinkStateBackend, address.getId(), namespace, ListCoder.of(coder));
-
-      this.namespace = namespace;
-      this.address = address;
-    }
-
-    @Override
-    public void add(T input) {
-      List<T> list = readInternal();
-      if (list == null) {
-        list = new ArrayList<>();
-      }
-      list.add(input);
-      writeInternal(list);
-    }
-
-    @Override
-    public BagState<T> readLater() {
-      return this;
-    }
-
-    @Override
-    public Iterable<T> read() {
-      List<T> result = readInternal();
-      return result != null ? result : Collections.<T>emptyList();
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            List<T> result = readInternal();
-            return result == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkBroadcastBagState<?, ?> that = (FlinkBroadcastBagState<?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private class FlinkCombiningState<K, InputT, AccumT, OutputT>
-      extends AbstractBroadcastState<AccumT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
-
-    FlinkCombiningState(
-        DefaultOperatorStateBackend flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder) {
-      super(flinkStateBackend, address.getId(), namespace, accumCoder);
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      AccumT current = readInternal();
-      if (current == null) {
-        current = combineFn.createAccumulator();
-      }
-      current = combineFn.addInput(current, value);
-      writeInternal(current);
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      AccumT current = readInternal();
-
-      if (current == null) {
-        writeInternal(accum);
-      } else {
-        current = combineFn.mergeAccumulators(Arrays.asList(current, accum));
-        writeInternal(current);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      return readInternal();
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(accumulators);
-    }
-
-    @Override
-    public OutputT read() {
-      AccumT accum = readInternal();
-      if (accum != null) {
-        return combineFn.extractOutput(accum);
-      } else {
-        return combineFn.extractOutput(combineFn.createAccumulator());
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return readInternal() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkCombiningState<?, ?, ?, ?> that =
-          (FlinkCombiningState<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private class FlinkKeyedCombiningState<K, InputT, AccumT, OutputT>
-      extends AbstractBroadcastState<AccumT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn;
-    private final FlinkBroadcastStateInternals<K> flinkStateInternals;
-
-    FlinkKeyedCombiningState(
-        DefaultOperatorStateBackend flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder,
-        FlinkBroadcastStateInternals<K> flinkStateInternals) {
-      super(flinkStateBackend, address.getId(), namespace, accumCoder);
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-      this.flinkStateInternals = flinkStateInternals;
-
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      try {
-        AccumT current = readInternal();
-        if (current == null) {
-          current = combineFn.createAccumulator(flinkStateInternals.getKey());
-        }
-        current = combineFn.addInput(flinkStateInternals.getKey(), current, value);
-        writeInternal(current);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state." , e);
-      }
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      try {
-        AccumT current = readInternal();
-        if (current == null) {
-          writeInternal(accum);
-        } else {
-          current = combineFn.mergeAccumulators(
-              flinkStateInternals.getKey(),
-              Arrays.asList(current, accum));
-          writeInternal(current);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state.", e);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      try {
-        return readInternal();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators);
-    }
-
-    @Override
-    public OutputT read() {
-      try {
-        AccumT accum = readInternal();
-        if (accum != null) {
-          return combineFn.extractOutput(flinkStateInternals.getKey(), accum);
-        } else {
-          return combineFn.extractOutput(
-              flinkStateInternals.getKey(),
-              combineFn.createAccumulator(flinkStateInternals.getKey()));
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return readInternal() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkKeyedCombiningState<?, ?, ?, ?> that =
-          (FlinkKeyedCombiningState<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-  private class FlinkCombiningStateWithContext<K, InputT, AccumT, OutputT>
-      extends AbstractBroadcastState<AccumT>
-      implements CombiningState<InputT, AccumT, OutputT> {
-
-    private final StateNamespace namespace;
-    private final StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address;
-    private final CombineWithContext.KeyedCombineFnWithContext<
-        ? super K, InputT, AccumT, OutputT> combineFn;
-    private final FlinkBroadcastStateInternals<K> flinkStateInternals;
-    private final CombineWithContext.Context context;
-
-    FlinkCombiningStateWithContext(
-        DefaultOperatorStateBackend flinkStateBackend,
-        StateTag<? super K, CombiningState<InputT, AccumT, OutputT>> address,
-        CombineWithContext.KeyedCombineFnWithContext<
-            ? super K, InputT, AccumT, OutputT> combineFn,
-        StateNamespace namespace,
-        Coder<AccumT> accumCoder,
-        FlinkBroadcastStateInternals<K> flinkStateInternals,
-        CombineWithContext.Context context) {
-      super(flinkStateBackend, address.getId(), namespace, accumCoder);
-
-      this.namespace = namespace;
-      this.address = address;
-      this.combineFn = combineFn;
-      this.flinkStateInternals = flinkStateInternals;
-      this.context = context;
-
-    }
-
-    @Override
-    public CombiningState<InputT, AccumT, OutputT> readLater() {
-      return this;
-    }
-
-    @Override
-    public void add(InputT value) {
-      try {
-        AccumT current = readInternal();
-        if (current == null) {
-          current = combineFn.createAccumulator(flinkStateInternals.getKey(), context);
-        }
-        current = combineFn.addInput(flinkStateInternals.getKey(), current, value, context);
-        writeInternal(current);
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state." , e);
-      }
-    }
-
-    @Override
-    public void addAccum(AccumT accum) {
-      try {
-
-        AccumT current = readInternal();
-        if (current == null) {
-          writeInternal(accum);
-        } else {
-          current = combineFn.mergeAccumulators(
-              flinkStateInternals.getKey(),
-              Arrays.asList(current, accum),
-              context);
-          writeInternal(current);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException("Error adding to state.", e);
-      }
-    }
-
-    @Override
-    public AccumT getAccum() {
-      try {
-        return readInternal();
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
-      return combineFn.mergeAccumulators(flinkStateInternals.getKey(), accumulators, context);
-    }
-
-    @Override
-    public OutputT read() {
-      try {
-        AccumT accum = readInternal();
-        return combineFn.extractOutput(flinkStateInternals.getKey(), accum, context);
-      } catch (Exception e) {
-        throw new RuntimeException("Error reading state.", e);
-      }
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty() {
-      return new ReadableState<Boolean>() {
-        @Override
-        public Boolean read() {
-          try {
-            return readInternal() == null;
-          } catch (Exception e) {
-            throw new RuntimeException("Error reading state.", e);
-          }
-
-        }
-
-        @Override
-        public ReadableState<Boolean> readLater() {
-          return this;
-        }
-      };
-    }
-
-    @Override
-    public void clear() {
-      clearInternal();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      FlinkCombiningStateWithContext<?, ?, ?, ?> that =
-          (FlinkCombiningStateWithContext<?, ?, ?, ?>) o;
-
-      return namespace.equals(that.namespace) && address.equals(that.address);
-
-    }
-
-    @Override
-    public int hashCode() {
-      int result = namespace.hashCode();
-      result = 31 * result + address.hashCode();
-      return result;
-    }
-  }
-
-}