You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2023/01/02 09:28:49 UTC

[GitHub] [beam] je-ik commented on a diff in pull request #24837: Enable async processing for SDF on Spark runner #23852

je-ik commented on code in PR #24837:
URL: https://github.com/apache/beam/pull/24837#discussion_r1059907930


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java:
##########
@@ -84,4 +84,11 @@ static void prepareFilesToStage(SparkCommonPipelineOptions options) {
       PipelineResources.prepareFilesForStaging(options);
     }
   }
+
+  @Description(
+      "Enable/disable async output for operators with possibly large output ( such as splittable DoFn )")
+  @Default.Boolean(true)
+  Boolean getEnableAsyncOperatorOutput();

Review Comment:
   Is this really what it does? Is it async? My understanding was, that it only uses two threads that pass data using BlockingQueue. Did I miss something?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {

Review Comment:
   Should this check if the bundle is non-empty? I.e. call startBundle only there are elements to be processed?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {
+          isBundleStarted = true;
+          // call startBundle() before beginning to process the partition.
+          ctx.getDoFnRunner().startBundle();
+        }
+
+        while (true) {
+          if (outputIterator.hasNext()) {
+            return outputIterator.next();
+          }
+
+          outputManager.clear();
+          if (inputIterator.hasNext()) {
+            // grab the next element and process it.
+            ctx.getDoFnRunner().processElement(inputIterator.next());
+            outputIterator = outputManager.iterator();
+          } else if (ctx.getTimerDataIterator().hasNext()) {
+            fireTimer(ctx.getTimerDataIterator().next());
+            outputIterator = outputManager.iterator();
+          } else {
+            // no more input to consume, but finishBundle can produce more output
+            if (!isBundleFinished) {
+              isBundleFinished = true;
+              ctx.getDoFnRunner().finishBundle();
+              outputIterator = outputManager.iterator();
+              continue; // try to consume outputIterator from start of loop
+            }
+            DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+            return endOfData();
+          }
+        }
+      } catch (final RuntimeException re) {
+        DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+        throw re;
+      }
+    }
+
+    private void fireTimer(TimerInternals.TimerData timer) {
+      StateNamespace namespace = timer.getNamespace();
+      checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+      BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+      ctx.getDoFnRunner()
+          .onTimer(
+              timer.getTimerId(),
+              timer.getTimerFamilyId(),
+              ctx.getKey(),
+              window,
+              timer.getTimestamp(),
+              timer.getOutputTimestamp(),
+              timer.getDomain());
+    }
+  }
+}
+
+class AsyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final BlockingDoFnOutputManager outputManager = new BlockingDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new AsyncInOutIterator<>(input, ctx);
+  }
+
+  /**
+   * Output manager which can hold limited number of output elements. If capacity is reached, then
+   * attempt to output more elements will block until some elements are consumed.
+   */
+  private class BlockingDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final LinkedBlockingQueue<Tuple2<TupleTag<?>, WindowedValue<?>>> queue =
+        new LinkedBlockingQueue<>(500);
+    private volatile boolean stopped = false;
+
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      return new Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+
+        @Override
+        public boolean hasNext() {
+          // expect elements appearing in queue until stop() is invoked.
+          // after that, no more inputs can arrive, so just drain the queue
+          while (true) {
+            if (queue.isEmpty()) {
+              try {
+                // Wait for a bit before checking again if more data is available
+                Thread.sleep(20);
+                if (stopped) {
+                  return !queue.isEmpty();
+                }
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+              }
+            } else {
+              return true;
+            }
+          }
+        }
+
+        @Override
+        public Tuple2<TupleTag<?>, WindowedValue<?>> next() {
+          Tuple2<TupleTag<?>, WindowedValue<?>> poll;
+          try {
+            poll = queue.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(e);

Review Comment:
   Possibly `IllegalStateException`?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java:
##########
@@ -84,4 +84,11 @@ static void prepareFilesToStage(SparkCommonPipelineOptions options) {
       PipelineResources.prepareFilesForStaging(options);
     }
   }
+
+  @Description(
+      "Enable/disable async output for operators with possibly large output ( such as splittable DoFn )")

Review Comment:
   ```suggestion
         "Enable/disable async output for operators with possibly large output (such as splittable DoFn)")
   ```



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;

Review Comment:
   ```suggestion
   ```



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {
+          isBundleStarted = true;
+          // call startBundle() before beginning to process the partition.
+          ctx.getDoFnRunner().startBundle();
+        }
+
+        while (true) {
+          if (outputIterator.hasNext()) {
+            return outputIterator.next();
+          }
+
+          outputManager.clear();
+          if (inputIterator.hasNext()) {
+            // grab the next element and process it.
+            ctx.getDoFnRunner().processElement(inputIterator.next());
+            outputIterator = outputManager.iterator();
+          } else if (ctx.getTimerDataIterator().hasNext()) {
+            fireTimer(ctx.getTimerDataIterator().next());
+            outputIterator = outputManager.iterator();
+          } else {
+            // no more input to consume, but finishBundle can produce more output
+            if (!isBundleFinished) {
+              isBundleFinished = true;
+              ctx.getDoFnRunner().finishBundle();
+              outputIterator = outputManager.iterator();
+              continue; // try to consume outputIterator from start of loop
+            }
+            DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+            return endOfData();
+          }
+        }
+      } catch (final RuntimeException re) {
+        DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+        throw re;
+      }
+    }
+
+    private void fireTimer(TimerInternals.TimerData timer) {
+      StateNamespace namespace = timer.getNamespace();
+      checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+      BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+      ctx.getDoFnRunner()
+          .onTimer(
+              timer.getTimerId(),
+              timer.getTimerFamilyId(),
+              ctx.getKey(),
+              window,
+              timer.getTimestamp(),
+              timer.getOutputTimestamp(),
+              timer.getDomain());
+    }
+  }
+}
+
+class AsyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final BlockingDoFnOutputManager outputManager = new BlockingDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new AsyncInOutIterator<>(input, ctx);
+  }
+
+  /**
+   * Output manager which can hold limited number of output elements. If capacity is reached, then
+   * attempt to output more elements will block until some elements are consumed.
+   */
+  private class BlockingDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final LinkedBlockingQueue<Tuple2<TupleTag<?>, WindowedValue<?>>> queue =
+        new LinkedBlockingQueue<>(500);
+    private volatile boolean stopped = false;
+
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      return new Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+
+        @Override
+        public boolean hasNext() {
+          // expect elements appearing in queue until stop() is invoked.
+          // after that, no more inputs can arrive, so just drain the queue
+          while (true) {
+            if (queue.isEmpty()) {
+              try {
+                // Wait for a bit before checking again if more data is available
+                Thread.sleep(20);

Review Comment:
   Why to have fixed waiting? Could it do `queue.poll(timeout)`?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {
+          isBundleStarted = true;
+          // call startBundle() before beginning to process the partition.
+          ctx.getDoFnRunner().startBundle();
+        }
+
+        while (true) {
+          if (outputIterator.hasNext()) {
+            return outputIterator.next();
+          }
+
+          outputManager.clear();
+          if (inputIterator.hasNext()) {
+            // grab the next element and process it.
+            ctx.getDoFnRunner().processElement(inputIterator.next());
+            outputIterator = outputManager.iterator();
+          } else if (ctx.getTimerDataIterator().hasNext()) {
+            fireTimer(ctx.getTimerDataIterator().next());
+            outputIterator = outputManager.iterator();
+          } else {
+            // no more input to consume, but finishBundle can produce more output
+            if (!isBundleFinished) {
+              isBundleFinished = true;
+              ctx.getDoFnRunner().finishBundle();
+              outputIterator = outputManager.iterator();
+              continue; // try to consume outputIterator from start of loop
+            }
+            DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+            return endOfData();
+          }
+        }
+      } catch (final RuntimeException re) {
+        DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+        throw re;
+      }
+    }
+
+    private void fireTimer(TimerInternals.TimerData timer) {
+      StateNamespace namespace = timer.getNamespace();
+      checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+      BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+      ctx.getDoFnRunner()
+          .onTimer(
+              timer.getTimerId(),
+              timer.getTimerFamilyId(),
+              ctx.getKey(),
+              window,
+              timer.getTimestamp(),
+              timer.getOutputTimestamp(),
+              timer.getDomain());
+    }
+  }
+}
+
+class AsyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final BlockingDoFnOutputManager outputManager = new BlockingDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new AsyncInOutIterator<>(input, ctx);
+  }
+
+  /**
+   * Output manager which can hold limited number of output elements. If capacity is reached, then
+   * attempt to output more elements will block until some elements are consumed.
+   */
+  private class BlockingDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final LinkedBlockingQueue<Tuple2<TupleTag<?>, WindowedValue<?>>> queue =
+        new LinkedBlockingQueue<>(500);
+    private volatile boolean stopped = false;
+
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      return new Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+
+        @Override
+        public boolean hasNext() {
+          // expect elements appearing in queue until stop() is invoked.
+          // after that, no more inputs can arrive, so just drain the queue
+          while (true) {
+            if (queue.isEmpty()) {
+              try {
+                // Wait for a bit before checking again if more data is available
+                Thread.sleep(20);
+                if (stopped) {
+                  return !queue.isEmpty();
+                }
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();

Review Comment:
   This should break the loop. Might be safer to rethrow the exception, because terminating the iteration looks dangerous.



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {
+          isBundleStarted = true;
+          // call startBundle() before beginning to process the partition.
+          ctx.getDoFnRunner().startBundle();
+        }
+
+        while (true) {
+          if (outputIterator.hasNext()) {
+            return outputIterator.next();
+          }
+
+          outputManager.clear();
+          if (inputIterator.hasNext()) {
+            // grab the next element and process it.
+            ctx.getDoFnRunner().processElement(inputIterator.next());
+            outputIterator = outputManager.iterator();
+          } else if (ctx.getTimerDataIterator().hasNext()) {
+            fireTimer(ctx.getTimerDataIterator().next());
+            outputIterator = outputManager.iterator();
+          } else {
+            // no more input to consume, but finishBundle can produce more output
+            if (!isBundleFinished) {
+              isBundleFinished = true;
+              ctx.getDoFnRunner().finishBundle();
+              outputIterator = outputManager.iterator();
+              continue; // try to consume outputIterator from start of loop
+            }
+            DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+            return endOfData();
+          }
+        }
+      } catch (final RuntimeException re) {
+        DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+        throw re;
+      }
+    }
+
+    private void fireTimer(TimerInternals.TimerData timer) {
+      StateNamespace namespace = timer.getNamespace();
+      checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+      BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+      ctx.getDoFnRunner()
+          .onTimer(
+              timer.getTimerId(),
+              timer.getTimerFamilyId(),
+              ctx.getKey(),
+              window,
+              timer.getTimestamp(),
+              timer.getOutputTimestamp(),
+              timer.getDomain());
+    }
+  }
+}
+
+class AsyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final BlockingDoFnOutputManager outputManager = new BlockingDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new AsyncInOutIterator<>(input, ctx);
+  }
+
+  /**
+   * Output manager which can hold limited number of output elements. If capacity is reached, then
+   * attempt to output more elements will block until some elements are consumed.
+   */
+  private class BlockingDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final LinkedBlockingQueue<Tuple2<TupleTag<?>, WindowedValue<?>>> queue =
+        new LinkedBlockingQueue<>(500);
+    private volatile boolean stopped = false;
+
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      return new Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+
+        @Override
+        public boolean hasNext() {
+          // expect elements appearing in queue until stop() is invoked.
+          // after that, no more inputs can arrive, so just drain the queue
+          while (true) {
+            if (queue.isEmpty()) {
+              try {
+                // Wait for a bit before checking again if more data is available
+                Thread.sleep(20);
+                if (stopped) {
+                  return !queue.isEmpty();
+                }
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+              }
+            } else {
+              return true;
+            }
+          }
+        }
+
+        @Override
+        public Tuple2<TupleTag<?>, WindowedValue<?>> next() {

Review Comment:
   Do we lack bundle management here? I.e. starting bundle, finishing bundle, teardown?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.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.spark.translation;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+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.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import scala.Tuple2;
+
+/** Processes Spark partitions using Beam's {@link org.apache.beam.runners.core.DoFnRunner}. */
+interface SparkInputDataProcessor<FnInputT, FnOutputT, OutputT> {
+
+  /**
+   * @return {@link OutputManager} to be used by {@link org.apache.beam.runners.core.DoFnRunner} for
+   *     emitting processing results
+   */
+  OutputManager getOutputManager();
+
+  /**
+   * Processes input partition data and return results as {@link Iterable}.
+   *
+   * @param input input partition iterator
+   * @param ctx current processing context
+   */
+  <K> Iterable<OutputT> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx);
+
+  /**
+   * Creates a synchronous {@link SparkInputDataProcessor} which does processing in calling thread.
+   * It is doing so by processing fully input element and then iterating over the output retrieved
+   * from that processing. The result of processing one element fit into memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createSync() {
+    return new SyncSparkInputDataProcessor<>();
+  }
+
+  /**
+   * Create and asynchronous {@link SparkInputDataProcessor} which does process input elements in
+   * separate thread and observes produced outputs asynchronously. This does not require results of
+   * processing one element to fit into the memory.
+   */
+  static <FnInputT, FnOutputT>
+      SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>>
+          createAsync() {
+    return new AsyncSparkInputDataProcessor<>();
+  }
+}
+
+class SyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final UnboundedDoFnOutputManager outputManager = new UnboundedDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new SyncInOutIterator<>(input, ctx);
+  }
+
+  private static class UnboundedDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
+
+    public void clear() {
+      outputs.clear();
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
+      return Iterators.transform(entryIter, this.entryToTupleFn());
+    }
+
+    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
+      return en -> {
+        if (en == null) {
+          return null;
+        } else {
+          return new Tuple2<>(en.getKey(), en.getValue());
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      outputs.put(tag, output);
+    }
+  }
+
+  private class SyncInOutIterator<K>
+      extends AbstractIterator<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final Iterator<WindowedValue<FnInputT>> inputIterator;
+    private final SparkProcessContext<K, FnInputT, FnOutputT> ctx;
+    private Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> outputIterator;
+    private boolean isBundleStarted;
+    private boolean isBundleFinished;
+
+    SyncInOutIterator(
+        Iterator<WindowedValue<FnInputT>> iterator,
+        SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+      this.inputIterator = iterator;
+      this.ctx = ctx;
+      this.outputIterator = outputManager.iterator();
+      ;
+    }
+
+    @Override
+    protected Tuple2<TupleTag<?>, WindowedValue<?>> computeNext() {
+      try {
+        // Process each element from the (input) iterator, which produces, zero, one or more
+        // output elements (of type V) in the output iterator. Note that the output
+        // collection (and iterator) is reset between each call to processElement, so the
+        // collection only holds the output values for each call to processElement, rather
+        // than for the whole partition (which would use too much memory).
+        if (!isBundleStarted) {
+          isBundleStarted = true;
+          // call startBundle() before beginning to process the partition.
+          ctx.getDoFnRunner().startBundle();
+        }
+
+        while (true) {
+          if (outputIterator.hasNext()) {
+            return outputIterator.next();
+          }
+
+          outputManager.clear();
+          if (inputIterator.hasNext()) {
+            // grab the next element and process it.
+            ctx.getDoFnRunner().processElement(inputIterator.next());
+            outputIterator = outputManager.iterator();
+          } else if (ctx.getTimerDataIterator().hasNext()) {
+            fireTimer(ctx.getTimerDataIterator().next());
+            outputIterator = outputManager.iterator();
+          } else {
+            // no more input to consume, but finishBundle can produce more output
+            if (!isBundleFinished) {
+              isBundleFinished = true;
+              ctx.getDoFnRunner().finishBundle();
+              outputIterator = outputManager.iterator();
+              continue; // try to consume outputIterator from start of loop
+            }
+            DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+            return endOfData();
+          }
+        }
+      } catch (final RuntimeException re) {
+        DoFnInvokers.invokerFor(ctx.getDoFn()).invokeTeardown();
+        throw re;
+      }
+    }
+
+    private void fireTimer(TimerInternals.TimerData timer) {
+      StateNamespace namespace = timer.getNamespace();
+      checkArgument(namespace instanceof StateNamespaces.WindowNamespace);
+      BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow();
+      ctx.getDoFnRunner()
+          .onTimer(
+              timer.getTimerId(),
+              timer.getTimerFamilyId(),
+              ctx.getKey(),
+              window,
+              timer.getTimestamp(),
+              timer.getOutputTimestamp(),
+              timer.getDomain());
+    }
+  }
+}
+
+class AsyncSparkInputDataProcessor<FnInputT, FnOutputT>
+    implements SparkInputDataProcessor<FnInputT, FnOutputT, Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+  private final BlockingDoFnOutputManager outputManager = new BlockingDoFnOutputManager();
+
+  @Override
+  public OutputManager getOutputManager() {
+    return outputManager;
+  }
+
+  @Override
+  public <K> Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> process(
+      Iterator<WindowedValue<FnInputT>> input, SparkProcessContext<K, FnInputT, FnOutputT> ctx) {
+    if (!input.hasNext()) {
+      return Collections.emptyList();
+    }
+    return () -> new AsyncInOutIterator<>(input, ctx);
+  }
+
+  /**
+   * Output manager which can hold limited number of output elements. If capacity is reached, then
+   * attempt to output more elements will block until some elements are consumed.
+   */
+  private class BlockingDoFnOutputManager
+      implements OutputManager, Iterable<Tuple2<TupleTag<?>, WindowedValue<?>>> {
+
+    private final LinkedBlockingQueue<Tuple2<TupleTag<?>, WindowedValue<?>>> queue =
+        new LinkedBlockingQueue<>(500);
+    private volatile boolean stopped = false;
+
+    public void stop() {
+      stopped = true;
+    }
+
+    @Override
+    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
+      return new Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>>() {
+
+        @Override
+        public boolean hasNext() {
+          // expect elements appearing in queue until stop() is invoked.
+          // after that, no more inputs can arrive, so just drain the queue
+          while (true) {
+            if (queue.isEmpty()) {
+              try {
+                // Wait for a bit before checking again if more data is available
+                Thread.sleep(20);
+                if (stopped) {
+                  return !queue.isEmpty();
+                }
+              } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+              }
+            } else {
+              return true;
+            }
+          }
+        }
+
+        @Override
+        public Tuple2<TupleTag<?>, WindowedValue<?>> next() {
+          Tuple2<TupleTag<?>, WindowedValue<?>> poll;
+          try {
+            poll = queue.poll(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(e);
+          }
+          return poll;
+        }
+      };
+    }
+
+    @Override
+    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      try {
+        if (stopped) {
+          throw new IllegalStateException("Output called on already stopped manager");
+        }
+        queue.put(new Tuple2<>(tag, output));
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();

Review Comment:
   I think we should rethrow here as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org