From 21d355ab471e961f5be5bdf3de2771cea18cde73 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 2 May 2016 13:25:40 -0700 Subject: [PATCH 01/29] beam-wide: blacklist Throwables.propagate and remove uses This is a forward-port of https://github.com/GoogleCloudPlatform/DataflowJavaSDK/pull/232 --- .../beam/examples/common/DataflowExampleUtils.java | 3 +-- runners/direct-java/pom.xml | 14 -------------- .../direct/ImmutabilityCheckingBundleFactory.java | 3 +-- .../runners/direct/InProcessPipelineRunner.java | 6 ++++-- .../direct/InProcessSideInputContainer.java | 3 +-- .../beam/runners/direct/TransformExecutor.java | 7 ++++--- .../beam/runners/dataflow/DataflowPipelineJob.java | 3 +-- .../src/main/resources/beam/checkstyle.xml | 6 ++++++ .../main/java/org/apache/beam/sdk/io/PubsubIO.java | 3 +-- .../beam/sdk/options/PipelineOptionsFactory.java | 4 ++-- .../apache/beam/sdk/transforms/DoFnReflector.java | 3 +-- .../sdk/transforms/IntraBundleParallelization.java | 4 ++-- .../beam/sdk/util/BigQueryTableInserter.java | 3 +-- .../org/apache/beam/sdk/util/ReduceFnTester.java | 3 +-- .../org/apache/beam/sdk/util/TriggerTester.java | 4 +--- .../java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 3 +-- 16 files changed, 28 insertions(+), 44 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java index 7ac71d300b99..fb4f3bfb413c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java @@ -51,7 +51,6 @@ import com.google.api.services.pubsub.model.Subscription; import com.google.api.services.pubsub.model.Topic; import com.google.common.base.Strings; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; @@ -116,7 +115,7 @@ public void setup() throws IOException { Thread.currentThread().interrupt(); // Ignore InterruptedException } - Throwables.propagate(lastException); + throw new RuntimeException(lastException); } /** diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 12ba329db70c..15a4e9cdd100 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -278,20 +278,6 @@ ${project.version} - - com.google.http-client - google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - - - com.google.http-client google-http-client-protobuf diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index 2103ad3b2b91..3b3821199beb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; -import com.google.api.client.util.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.SetMultimap; @@ -100,7 +99,7 @@ public UncommittedBundle add(WindowedValue element) { mutationDetectors.put( element, MutationDetectors.forValueWithCoder(element.getValue(), coder)); } catch (CoderException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } underlying.add(element); return this; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java index bb8c0de68613..19e9f47de568 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -256,7 +255,10 @@ public InProcessPipelineResult run(Pipeline pipeline) { } catch (UserCodeException userException) { throw new PipelineExecutionException(userException.getCause()); } catch (Throwable t) { - Throwables.propagate(t); + if (t instanceof RuntimeException) { + throw (RuntimeException) t; + } + throw new RuntimeException(t); } } return result; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java index f4980ef1546c..b01cd484c2e2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.values.PCollectionView; import com.google.common.base.MoreObjects; -import com.google.common.base.Throwables; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -172,7 +171,7 @@ private void updatePCollectionViewWindowValues( future.set(Collections.>emptyList()); } } catch (ExecutionException e) { - Throwables.propagate(e.getCause()); + throw new RuntimeException(e.getCause()); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index 8346e89f5af4..9e15c2aab830 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -23,8 +23,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; -import com.google.common.base.Throwables; - import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; @@ -119,7 +117,10 @@ public InProcessTransformResult call() { return result; } catch (Throwable t) { onComplete.handleThrowable(inputBundle, t); - throw Throwables.propagate(t); + if (t instanceof RuntimeException) { + throw (RuntimeException) t; + } + throw new RuntimeException(t); } finally { transformEvaluationState.complete(this); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 69565ac6573e..0f42148a0a4c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -41,7 +41,6 @@ import com.google.api.services.dataflow.model.JobMetrics; import com.google.api.services.dataflow.model.MetricUpdate; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -356,7 +355,7 @@ private boolean nextBackOff(Sleeper sleeper, BackOff backoff) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - throw Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml index 31717ffda967..2a4f8327fec2 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml @@ -59,6 +59,12 @@ page at http://checkstyle.sourceforge.net/config.html --> + + + + + + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 66d1d4335505..46464614221a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -51,7 +51,6 @@ import com.google.api.services.pubsub.model.ReceivedMessage; import com.google.api.services.pubsub.model.Subscription; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import org.joda.time.Duration; @@ -814,7 +813,7 @@ public void processElement(ProcessContext c) throws IOException { } } if (finallyBlockException != null) { - Throwables.propagate(finallyBlockException); + throw new RuntimeException(finallyBlockException); } for (PubsubMessage message : messages) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 5fc7312d8ef3..5f2dd1129458 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -32,7 +33,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Strings; -import com.google.common.base.Throwables; import com.google.common.collect.Collections2; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableListMultimap; @@ -991,7 +991,7 @@ private static List validateClass(Class void invoke(Method m, throw UserCodeException.wrap(e.getCause()); } catch (IllegalAccessException | IllegalArgumentException e) { // Exception in our code. - throw Throwables.propagate(e); + throw new RuntimeException(e); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java index 4b3afb426ea0..62c09c2158ac 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java @@ -221,7 +221,7 @@ public void processElement(final ProcessContext c) throws Exception { } if (failure.get() != null) { - throw Throwables.propagate(failure.get()); + throw new RuntimeException(failure.get()); } executor.submit(new Runnable() { @@ -246,7 +246,7 @@ public void finishBundle(Context c) throws Exception { // processElement calls have finished. workTickets.acquire(maxParallelism); if (failure.get() != null) { - throw Throwables.propagate(failure.get()); + throw new RuntimeException(failure.get()); } doFn.finishBundle(c); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java index 86a0b5befca8..0493f1c7fa49 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java @@ -37,7 +37,6 @@ import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; @@ -275,7 +274,7 @@ public List call() throws IOException { Thread.currentThread().interrupt(); throw new IOException("Interrupted while inserting " + rowsToPublish); } catch (ExecutionException e) { - Throwables.propagate(e.getCause()); + throw new RuntimeException(e.getCause()); } if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java index f0d2a44a1ae9..f296d653074b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java @@ -58,7 +58,6 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -408,7 +407,7 @@ public WindowedValue apply(TimestampedValue input) { windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE))); return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } })); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java index 10d3d3521ca5..0889b4f66ca7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -45,7 +44,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -257,7 +255,7 @@ public final void injectElements(Collection> values) th windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); } catch (Exception e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index e6053116fbe5..4f353df865ff 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -44,7 +44,6 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Throwables; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -634,7 +633,7 @@ public UnboundedKafkaReader createReader(PipelineOptions options, return new UnboundedKafkaReader( generateInitialSplits(1, options).get(0), checkpointMark); } catch (Exception e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } return new UnboundedKafkaReader(this, checkpointMark); From 003a300eb352a3ba09980711814d0ba125e95e69 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 2 May 2016 15:26:39 -0700 Subject: [PATCH 02/29] Fix the java doc for Combine.perKey and ApproximateQuantiles --- .../org/apache/beam/sdk/transforms/ApproximateQuantiles.java | 4 ++-- .../src/main/java/org/apache/beam/sdk/transforms/Combine.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index 66f7a6a327a8..4eb06d22b87a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -74,7 +74,7 @@ private ApproximateQuantiles() { * {@code numQuantiles-2} intermediate values, and maximum value, in * sorted order, using the given {@code Comparator} to order values. * To compute traditional {@code N}-tiles, one should use - * {@code ApproximateQuantiles.globally(compareFn, N+1)}. + * {@code ApproximateQuantiles.globally(N+1, compareFn)}. * *

If there are fewer input elements than {@code numQuantiles}, * then the result {@code List} will contain all the input elements, @@ -86,7 +86,7 @@ private ApproximateQuantiles() { *

 {@code
    * PCollection pc = ...;
    * PCollection> quantiles =
-   *     pc.apply(ApproximateQuantiles.globally(stringCompareFn, 11));
+   *     pc.apply(ApproximateQuantiles.globally(11, stringCompareFn));
    * } 
* * @param the type of the elements in the input {@code PCollection} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index f3a7c0360100..8a39c98a16e7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1661,7 +1661,7 @@ protected SimpleCombineFn(SerializableFunction, V> combiner) { *
 {@code
    * PCollection> salesRecords = ...;
    * PCollection> totalSalesPerPerson =
-   *     salesRecords.apply(Combine.perKey(
+   *     salesRecords.apply(Combine.perKey(
    *         new Sum.SumDoubleFn()));
    * } 
* From 0e9041ae0f53b08879e569c1c0501ce5c4651720 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 28 Apr 2016 13:42:36 -0700 Subject: [PATCH 03/29] Use CommittedResult in InMemoryWatermarkManager This enable unprocessed elements to be handled in the Watermark manager after they are added to the CommittedResult structure. --- .../direct/InMemoryWatermarkManager.java | 19 +- .../direct/InProcessEvaluationContext.java | 6 +- .../direct/InMemoryWatermarkManagerTest.java | 368 ++++++++++++------ 3 files changed, 252 insertions(+), 141 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java index 769457a77a8c..4d5a3a13f605 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java @@ -800,18 +800,19 @@ public TransformWatermarks getWatermarks(AppliedPTransform transform) { * . * * @param completed the input that has completed - * @param transform the transform that has completed processing the input - * @param outputs the bundles the transform has output + * @param timerUpdate the timers that were added, removed, and completed as part of producing + * this update + * @param result the result that was produced by processing the input * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there * is no hold */ public void updateWatermarks( @Nullable CommittedBundle completed, - AppliedPTransform transform, TimerUpdate timerUpdate, - Iterable> outputs, + CommittedResult result, @Nullable Instant earliestHold) { - updatePending(completed, transform, timerUpdate, outputs); + AppliedPTransform transform = result.getTransform(); + updatePending(completed, timerUpdate, result); TransformWatermarks transformWms = transformToWatermarks.get(transform); transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold); refreshWatermarks(transform); @@ -846,15 +847,14 @@ private void refreshWatermarks(AppliedPTransform transform) { */ private void updatePending( CommittedBundle input, - AppliedPTransform transform, TimerUpdate timerUpdate, - Iterable> outputs) { - TransformWatermarks completedTransform = transformToWatermarks.get(transform); + CommittedResult result) { + TransformWatermarks completedTransform = transformToWatermarks.get(result.getTransform()); // Newly pending elements must be added before completed elements are removed, as the two // do not share a Mutex within this call and thus can be interleaved with external calls to // refresh. - for (CommittedBundle bundle : outputs) { + for (CommittedBundle bundle : result.getOutputs()) { for (AppliedPTransform consumer : consumers.get(bundle.getPCollection())) { TransformWatermarks watermarks = transformToWatermarks.get(consumer); watermarks.addPending(bundle); @@ -865,7 +865,6 @@ private void updatePending( if (input != null) { completedTransform.removePending(input); } - } /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java index d9a7ff02f839..d4f891ee450c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java @@ -152,11 +152,11 @@ public synchronized CommittedResult handleResult( Iterable> committedBundles = commitBundles(result.getOutputBundles()); // Update watermarks and timers + CommittedResult committedResult = CommittedResult.create(result, committedBundles); watermarkManager.updateWatermarks( completedBundle, - result.getTransform(), result.getTimerUpdate().withCompletedTimers(completedTimers), - committedBundles, + committedResult, result.getWatermarkHold()); fireAllAvailableCallbacks(); // Update counters @@ -176,7 +176,7 @@ public synchronized CommittedResult handleResult( applicationStateInternals.remove(stepAndKey); } } - return CommittedResult.create(result, committedBundles); + return committedResult; } private Iterable> commitBundles( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java index 2880adec6832..15cdf8a32e06 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java @@ -159,8 +159,11 @@ public void getWatermarkForUntouchedTransform() { @Test public void getWatermarkForUpdatedSourceTransform() { CommittedBundle output = multiWindowedBundle(createdInts, 1); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(output), new Instant(8000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(output)), + new Instant(8000L)); TransformWatermarks updatedSourceWatermark = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -175,8 +178,10 @@ public void getWatermarkForUpdatedSourceTransform() { public void getWatermarkForMultiInputTransform() { CommittedBundle secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1); - manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(secondPcollectionBundle), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(intsToFlatten.getProducingTransformInternal(), + Collections.>singleton(secondPcollectionBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); // We didn't do anything for the first source, so we shouldn't have progressed the watermark @@ -205,13 +210,17 @@ public void getWatermarkForMultiInputTransform() { CommittedBundle flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1); // We have finished processing the bundle from the second PCollection, but we haven't consumed // anything from the first PCollection yet; so our watermark shouldn't advance - manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(flattenedBundleSecondCreate), + manager.updateWatermarks(secondPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(flattenedBundleSecondCreate)), null); TransformWatermarks transformAfterProcessing = manager.getWatermarks(flattened.getProducingTransformInternal()); - manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(flattenedBundleSecondCreate), + manager.updateWatermarks(secondPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(flattenedBundleSecondCreate)), null); assertThat( transformAfterProcessing.getInputWatermark(), @@ -225,8 +234,10 @@ public void getWatermarkForMultiInputTransform() { timestampedBundle(createdInts, TimestampedValue.of(5, firstCollectionTimestamp)); // the source is done, but elements are still buffered. The source output watermark should be // past the end of the global window - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstPcollectionBundle), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstPcollectionBundle)), new Instant(Long.MAX_VALUE)); TransformWatermarks firstSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -253,8 +264,10 @@ public void getWatermarkForMultiInputTransform() { CommittedBundle completedFlattenBundle = bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(completedFlattenBundle), + manager.updateWatermarks(firstPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(completedFlattenBundle)), null); TransformWatermarks afterConsumingAllInput = manager.getWatermarks(flattened.getProducingTransformInternal()); @@ -275,8 +288,11 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle createdBundle = timestampedBundle(createdInts, TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); TransformWatermarks createdAfterProducing = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat( @@ -287,8 +303,11 @@ public void getWatermarkForMultiConsumedCollection() { timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), + null); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); assertThat( @@ -303,8 +322,11 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle filteredBundle = timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L))); - manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(filteredBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), + null); TransformWatermarks filteredProcessedWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); assertThat( @@ -322,17 +344,23 @@ public void getWatermarkForMultiConsumedCollection() { @Test public void updateWatermarkWithWatermarkHolds() { CommittedBundle createdBundle = timestampedBundle(createdInts, - TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), + TimestampedValue.of(1, new Instant(1_000_000L)), + TimestampedValue.of(2, new Instant(1234L)), TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); - CommittedBundle> keyBundle = - timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), - TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), - TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), + CommittedBundle> keyBundle = timestampedBundle(keyed, + TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), + TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), + TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), new Instant(500L)); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -358,40 +386,54 @@ public void updateWatermarkWithKeyedWatermarkHolds() { .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L))) .commit(clock.now()); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + ImmutableList.of(firstKeyBundle, secondKeyBundle)), + BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(-1000L)); - manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(1234L)); + manager.updateWatermarks(firstKeyBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(-1000L)); + manager.updateWatermarks(secondKeyBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(1234L)); TransformWatermarks filteredWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); - assertThat( - filteredWatermarks.getInputWatermark(), + assertThat(filteredWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L)))); CommittedBundle fauxFirstKeyTimerBundle = bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now()); - manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(fauxFirstKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L))); CommittedBundle fauxSecondKeyTimerBundle = bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now()); - manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(5678L)); + manager.updateWatermarks(fauxSecondKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(5678L)); assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L))); - manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(fauxSecondKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); - assertThat( - filteredWatermarks.getOutputWatermark(), + assertThat(filteredWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); } @@ -403,16 +445,21 @@ public void updateWatermarkWithKeyedWatermarkHolds() { public void updateOutputWatermarkShouldBeMonotonic() { CommittedBundle firstInput = bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstInput), new Instant(0L)); + manager.updateWatermarks(null, TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstInput)), + new Instant(0L)); TransformWatermarks firstWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L))); CommittedBundle secondInput = bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(secondInput), new Instant(-250L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(secondInput)), + new Instant(-250L)); TransformWatermarks secondWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L)))); @@ -425,17 +472,22 @@ public void updateOutputWatermarkShouldBeMonotonic() { @Test public void updateWatermarkWithHoldsShouldBeMonotonic() { CommittedBundle createdBundle = timestampedBundle(createdInts, - TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), - TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + TimestampedValue.of(1, new Instant(1_000_000L)), + TimestampedValue.of(2, new Instant(1234L)), + TimestampedValue.of(3, new Instant(-1000L))); manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); CommittedBundle> keyBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), new Instant(500L)); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -462,16 +514,22 @@ public void updateWatermarkWithLateData() { CommittedBundle createdBundle = timestampedBundle(createdInts, TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), sourceWatermark); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + sourceWatermark); CommittedBundle> keyBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L))); // Finish processing the on-time data. The watermarks should progress to be equal to the source - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), + null); TransformWatermarks onTimeWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark)); @@ -481,8 +539,11 @@ public void updateWatermarkWithLateData() { timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L))); // the late data arrives in a downstream PCollection after its watermark has advanced past it; // we don't advance the watermark past the current watermark until we've consumed the late data - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(lateDataBundle), new Instant(2_000_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(lateDataBundle)), + new Instant(2_000_000L)); TransformWatermarks bufferedLateWm = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L))); @@ -496,30 +557,31 @@ public void updateWatermarkWithLateData() { CommittedBundle> lateKeyedBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(lateKeyedBundle), null); + manager.updateWatermarks(lateDataBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(lateKeyedBundle)), + null); } public void updateWatermarkWithDifferentWindowedValueInstances() { manager.updateWatermarks( null, - createdInts.getProducingTransformInternal(), TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.>singleton( bundleFactory .createRootBundle(createdInts) .add(WindowedValue.valueInGlobalWindow(1)) - .commit(Instant.now())), + .commit(Instant.now()))), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks( - bundleFactory - .createRootBundle(createdInts) + bundleFactory.createRootBundle(createdInts) .add(WindowedValue.valueInGlobalWindow(1)) .commit(Instant.now()), - keyed.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), + result(keyed.getProducingTransformInternal(), Collections.>emptyList()), null); TransformWatermarks onTimeWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -533,8 +595,10 @@ public void updateWatermarkWithDifferentWindowedValueInstances() { @Test public void getWatermarksAfterOnlyEmptyOutput() { CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(emptyCreateOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks updatedSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -560,12 +624,17 @@ public void getWatermarksAfterOnlyEmptyOutput() { @Test public void getWatermarksAfterHoldAndEmptyOutput() { CommittedBundle firstCreateOutput = multiWindowedBundle(createdInts, 1, 2); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstCreateOutput), new Instant(12_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstCreateOutput)), + new Instant(12_000L)); CommittedBundle firstFilterOutput = multiWindowedBundle(filtered); - manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(firstFilterOutput), + manager.updateWatermarks(firstCreateOutput, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(firstFilterOutput)), new Instant(10_000L)); TransformWatermarks firstFilterWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -573,8 +642,10 @@ public void getWatermarksAfterHoldAndEmptyOutput() { assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L)))); CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(emptyCreateOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks updatedSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -613,8 +684,11 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { CommittedBundle createOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createOutput)), + BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks createAfterUpdate = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now())); @@ -639,8 +713,10 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { CommittedBundle filterOutputBundle = bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L)); - manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(filterOutputBundle), + manager.updateWatermarks(createOutput, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filterOutputBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks filterAfterConsumed = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -661,8 +737,10 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { // @Test public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { CommittedBundle createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(1248L)); + manager.updateWatermarks(null, TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(1248L)); TransformWatermarks filteredWms = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -678,8 +756,10 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME); TimerUpdate timers = TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build(); - manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers, - Collections.>singleton(filteredBundle), + manager.updateWatermarks(createdBundle, + timers, + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); Instant startTime = clock.now(); clock.set(startTime.plus(250L)); @@ -712,11 +792,11 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo) .commit(filteredWms.getSynchronizedProcessingOutputTime()); // Complete the processing time timer - manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(), + manager.updateWatermarks(filteredTimerBundle, TimerUpdate.builder("key") - .withCompletedTimers(Collections.singleton(pastTimer)) - .build(), - Collections.>singleton(filteredTimerResult), + .withCompletedTimers(Collections.singleton(pastTimer)).build(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredTimerResult)), BoundedWindow.TIMESTAMP_MAX_VALUE); clock.set(startTime.plus(500L)); @@ -726,8 +806,10 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark()))); - manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(filteredTimerResult, + TimerUpdate.empty(), + result(filteredTimesTwo.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); @@ -761,18 +843,23 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { CommittedBundle createOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createOutput)), + BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks createAfterUpdate = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now()))); - assertThat( - createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now()))); + assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), + not(laterThan(clock.now()))); CommittedBundle createSecondOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(750L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createSecondOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createSecondOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); @@ -781,16 +868,20 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { @Test public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() { CommittedBundle created = multiWindowedBundle(createdInts, 1, 2, 3); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(created), new Instant(40_900L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(created)), + new Instant(40_900L)); CommittedBundle filteredBundle = multiWindowedBundle(filtered, 2, 4); Instant upstreamHold = new Instant(2048L); TimerData upstreamProcessingTimer = TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME); - manager.updateWatermarks(created, filtered.getProducingTransformInternal(), + manager.updateWatermarks(created, TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(), - Collections.>singleton(filteredBundle), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks downstreamWms = @@ -806,11 +897,12 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold)); CommittedBundle otherCreated = multiWindowedBundle(createdInts, 4, 8, 12); - manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(), + manager.updateWatermarks(otherCreated, TimerUpdate.builder("key") - .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)) - .build(), - Collections.>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE); + .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)).build(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now()))); } @@ -820,9 +912,9 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { CommittedBundle created = multiWindowedBundle(createdInts, 1, 2, 3); manager.updateWatermarks( null, - createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(created), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(created)), new Instant(29_919_235L)); Instant upstreamHold = new Instant(2048L); @@ -830,9 +922,9 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold); manager.updateWatermarks( created, - filtered.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(filteredBundle), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks downstreamWms = @@ -852,8 +944,10 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME); @@ -869,11 +963,10 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { .setTimer(lastTimer) .build(); - manager.updateWatermarks( - createdBundle, - filtered.getProducingTransformInternal(), + manager.updateWatermarks(createdBundle, update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -886,8 +979,11 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { FiredTimers firstFired = firstFilteredTimers.get(key); assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -909,8 +1005,10 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME); @@ -928,9 +1026,9 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { manager.updateWatermarks( createdBundle, - filtered.getProducingTransformInternal(), update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -944,8 +1042,11 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer)); clock.set(new Instant(50_000L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -967,8 +1068,10 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of( StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME); @@ -986,9 +1089,9 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { manager.updateWatermarks( createdBundle, - filtered.getProducingTransformInternal(), update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -1003,8 +1106,11 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer)); clock.set(new Instant(50_000L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -1133,7 +1239,6 @@ public boolean matches(Object item) { ReadableInstant instant = (ReadableInstant) item; return instant.isAfter(shouldBeEarlier); } - @Override public void describeTo(Description description) { description.appendText("later than ").appendValue(shouldBeEarlier); @@ -1165,4 +1270,11 @@ private final CommittedBundle multiWindowedBundle(PCollection pc, T... } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } + + private final CommittedResult result( + AppliedPTransform transform, + Iterable> bundles) { + return CommittedResult.create(StepTransformResult.withoutHold(transform) + .build(), bundles); + } } From 8797b140d7566fcb4e26afeb1ddb4206c5462550 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 3 May 2016 11:47:56 -0700 Subject: [PATCH 04/29] [BEAM-48] Upgrade bigquery library to v2-rev292-1.21.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 91c6da5ea4e1..88de395c7255 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,7 @@ 1.7.7 - v2-rev248-1.21.0 + v2-rev292-1.21.0 0.0.2 v2-rev6-1.21.0 v1b3-rev22-1.21.0 From b58741731176f2386f1b4e31fae72092209a3871 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 3 May 2016 13:07:04 -0700 Subject: [PATCH 05/29] [BEAM-255] Write: add limited logging This will help, for all sinks, users and developers gain insight into where time is spent. (Enabling DEBUG level will provide more insight.) --- .../main/java/org/apache/beam/sdk/io/Write.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index b6743fa93824..9cb026a9516c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -36,6 +36,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.UUID; /** @@ -52,6 +55,8 @@ */ @Experimental(Experimental.Kind.SOURCE_SINK) public class Write { + private static final Logger LOG = LoggerFactory.getLogger(Write.class); + /** * Creates a Write transform that writes to the given Sink. */ @@ -144,7 +149,9 @@ private PDone createWrite( @Override public void processElement(ProcessContext c) throws Exception { WriteOperation writeOperation = c.element(); + LOG.info("Initializing write operation {}", writeOperation); writeOperation.initialize(c.getPipelineOptions()); + LOG.debug("Done initializing write operation {}", writeOperation); // The WriteOperation is also the output of this ParDo, so it can have mutable // state. c.output(writeOperation); @@ -172,8 +179,10 @@ public void processElement(ProcessContext c) throws Exception { // Lazily initialize the Writer if (writer == null) { WriteOperation writeOperation = c.sideInput(writeOperationView); + LOG.info("Opening writer for write operation {}", writeOperation); writer = writeOperation.createWriter(c.getPipelineOptions()); writer.open(UUID.randomUUID().toString()); + LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView); } try { writer.write(c.element()); @@ -211,9 +220,12 @@ public void finishBundle(Context c) throws Exception { .apply("Finalize", ParDo.of(new DoFn, Integer>() { @Override public void processElement(ProcessContext c) throws Exception { - Iterable results = c.sideInput(resultsView); WriteOperation writeOperation = c.element(); + LOG.info("Finalizing write operation {}", writeOperation); + Iterable results = c.sideInput(resultsView); + LOG.debug("Side input initialized to finalize write operation {}", writeOperation); writeOperation.finalize(results, c.getPipelineOptions()); + LOG.debug("Done finalizing write operation {}", writeOperation); } }).withSideInputs(resultsView)); return PDone.in(input.getPipeline()); From cb71b563b92e4ed14fb84bb07056ad9a69986691 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 29 Apr 2016 15:26:01 +0200 Subject: [PATCH 06/29] [BEAM-154] Use dependencyManagement and pluginManagement to keep all modules sync in term of version --- examples/java/pom.xml | 96 ---- examples/java8/pom.xml | 84 +--- pom.xml | 449 +++++++++++++++++- runners/direct-java/pom.xml | 56 --- runners/flink/examples/pom.xml | 7 +- runners/flink/pom.xml | 64 +-- runners/flink/runner/pom.xml | 12 +- runners/google-cloud-dataflow-java/pom.xml | 123 ----- runners/spark/build-resources/checkstyle.xml | 225 --------- runners/spark/pom.xml | 163 +------ .../src/main/assembly}/src.xml | 0 sdks/java/core/pom.xml | 184 +------ sdks/java/extensions/join-library/pom.xml | 24 +- sdks/java/io/google-cloud-platform/pom.xml | 27 +- sdks/java/io/hdfs/pom.xml | 25 +- sdks/java/io/kafka/pom.xml | 34 +- sdks/java/java8tests/pom.xml | 38 -- 17 files changed, 554 insertions(+), 1057 deletions(-) delete mode 100644 runners/spark/build-resources/checkstyle.xml rename sdks/java/{ => build-tools/src/main/assembly}/src.xml (100%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 342986f3f906..3457f71aa96f 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -60,23 +60,6 @@ org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - @@ -150,7 +133,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 package @@ -182,20 +164,6 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -232,7 +200,6 @@ org.apache.beam java-sdk-all - ${project.version} @@ -244,121 +211,61 @@ com.google.api-client google-api-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-dataflow - ${dataflow.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - org.apache.avro avro - ${avro.version} com.google.apis google-api-services-datastore-protobuf - ${datastore.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.guava guava - ${guava.version} com.google.code.findbugs jsr305 - ${jsr305.version} joda-time joda-time - ${joda.version} org.slf4j slf4j-api - ${slf4j.version} org.slf4j slf4j-jdk14 - ${slf4j.version} runtime @@ -374,19 +281,16 @@ org.hamcrest hamcrest-all - ${hamcrest.version} junit junit - ${junit.version} org.mockito mockito-all - 1.10.19 test diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 95af76abc849..b4a9ec6d7fa4 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -71,42 +71,11 @@ org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -121,7 +90,6 @@ org.apache.beam java-sdk-all - ${project.version} @@ -139,114 +107,66 @@ com.google.guava guava - ${guava.version} org.slf4j slf4j-api - ${slf4j.version} org.apache.avro avro - ${avro.version} joda-time joda-time - ${joda.version} org.hamcrest hamcrest-all - ${hamcrest.version} test org.mockito mockito-all - 1.10.19 test junit junit - ${junit.version} test com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - + + diff --git a/pom.xml b/pom.xml index 88de395c7255..b0b258bb8318 100644 --- a/pom.xml +++ b/pom.xml @@ -107,13 +107,19 @@ v1b3-rev22-1.21.0 0.5.160222 v1beta2-rev1-4.0.0 + 1.0-rc2 + 1.1 1.21.0 + 1.4.3 + 0.5.160304 19.0 + 0.12.0 1.3 - 2.7.0 + 2.7.2 2.4 3.0.1 4.11 + 1.9.5 3.0.0-beta-1 v1-rev7-1.21.0 1.7.14 @@ -179,6 +185,364 @@ + + + + + org.apache.beam + java-sdk-all + ${project.version} + + + + org.apache.beam + google-cloud-dataflow-java-runner + ${project.version} + + + + org.apache.beam + java-examples-all + ${project.version} + + + + io.grpc + grpc-all + ${grpc.version} + + + + com.google.api-client + google-api-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.oauth-client + google-oauth-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.oauth-client + google-oauth-client-java6 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client-jackson + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + org.codehaus.jackson + jackson-core-asl + + + + + + com.google.http-client + google-http-client-jackson2 + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client-protobuf + ${google-clients.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-bigquery + ${bigquery.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-pubsub + ${pubsub.version} + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-storage + ${storage.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-datastore-protobuf + ${datastore.version} + + + + com.google.guava + guava-jdk5 + + + + com.google.http-client + google-http-client + + + com.google.api-client + google-api-client + + + com.google.oauth-client + google-oauth-client + + + com.google.http-client + google-http-client-jackson + + + com.google.http-client + google-http-client-protobuf + + + + + + com.google.auto.service + auto-service + ${google-auto-service.version} + + + + com.google.auto.value + auto-value + ${google-auto-value.version} + + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + + + + com.google.cloud.bigdataoss + util + ${google-cloud-bigdataoss.version} + + + com.google.cloud.bigdataoss + gcsio + ${google-cloud-bigdataoss.version} + + + + com.google.apis + google-api-services-dataflow + ${dataflow.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-clouddebugger + ${clouddebugger.version} + + + + com.google.guava + guava-jdk5 + + + + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.module + jackson-module-scala_2.10 + ${jackson.version} + + + + org.apache.avro + avro + ${avro.version} + + + + com.google.guava + guava + ${guava.version} + + + + joda-time + joda-time + ${joda.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + + + org.apache.beam + java-sdk-all + ${project.version} + test-jar + test + + + org.apache.beam + java-sdk-all + ${project.version} + tests + test + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + test + + + + junit + junit + ${junit.version} + test + + + + org.slf4j + slf4j-jdk14 + ${slf4j.version} + test + + + + com.google.guava + guava-testlib + ${guava.version} + test + + + + org.mockito + mockito-all + ${mockito.version} + test + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-proto-library-all + ${google-cloud-dataflow-java-proto-library-all.version} + test + + + + + @@ -249,12 +613,57 @@ org.apache.maven.plugins maven-jar-plugin 2.5 + + + default-jar + + jar + + + + default-test-jar + + test-jar + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + package + + jar-no-fork + + + + attach-test-sources + package + + test-jar-no-fork + + + org.apache.maven.plugins maven-javadoc-plugin 2.10.3 + + + javadoc + package + + jar + + + @@ -419,6 +828,18 @@ 1.10 + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + org.apache.maven.plugins maven-release-plugin @@ -432,6 +853,32 @@ + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.3.1 + + + enforce + + enforce + + + + + [1.7,) + + + [3.0.3,) + + + + + + + diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 15a4e9cdd100..292cc56c8c5e 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -69,43 +69,12 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - verify - - jar-no-fork - - - - attach-test-sources - verify - - test-jar-no-fork - - - @@ -194,7 +163,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 @@ -275,74 +243,55 @@ org.apache.beam java-sdk-all - ${project.version} com.google.http-client google-http-client-protobuf - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - runtime com.google.guava guava - - ${guava.version} com.google.guava guava-testlib - ${guava.version} test joda-time joda-time - ${joda.version} com.google.code.findbugs jsr305 - ${jsr305.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} org.slf4j slf4j-api - ${slf4j.version} com.google.auto.service auto-service - 1.0-rc2 true com.google.auto.value auto-value - 1.1 provided @@ -350,35 +299,30 @@ org.hamcrest hamcrest-all - ${hamcrest.version} provided junit junit - ${junit.version} provided org.slf4j slf4j-jdk14 - ${slf4j.version} test org.mockito mockito-all - 1.10.19 test org.apache.beam java-sdk-all - ${project.version} test-jar test diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index d27f2db2f361..e5bab3e9c03e 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -79,11 +79,16 @@ + org.codehaus.mojo exec-maven-plugin - 1.2.1 java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 608dc7c0020a..5e05e6a09f20 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -44,7 +44,6 @@ UTF-8 UTF-8 1.0.0 - 0.1.0-incubating-SNAPSHOT @@ -62,16 +61,27 @@ + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-source-plugin + + - - org.apache.maven.plugins maven-failsafe-plugin - 2.17 @@ -90,7 +100,6 @@ org.apache.maven.plugins maven-surefire-plugin - 2.17 -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit @@ -99,60 +108,15 @@ org.codehaus.mojo exec-maven-plugin - 1.2.1 none - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.7 - 1.7 - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.3.1 - - - enforce-maven - - enforce - - - - - [1.7,) - - - - [3.0.3,) - - - - - - - - - - diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 89c8d9949e66..ab8d266b28fb 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -91,7 +91,6 @@ org.apache.beam java-sdk-all - ${beam.version} org.slf4j @@ -103,7 +102,6 @@ org.apache.beam google-cloud-dataflow-java-runner - ${project.version} org.slf4j @@ -115,7 +113,6 @@ org.apache.beam java-examples-all - ${beam.version} org.slf4j @@ -140,14 +137,12 @@ org.mockito mockito-all - 1.9.5 test com.google.auto.service auto-service - 1.0-rc2 true @@ -155,6 +150,13 @@ + + org.apache.maven.plugins diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 55e7ed93145b..a1d48ce2e4d2 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -35,7 +35,6 @@ jar - 0.1.0-incubating-SNAPSHOT ${maven.build.timestamp} yyyy-MM-dd HH:mm @@ -108,43 +107,12 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - @@ -220,7 +188,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 @@ -318,194 +285,110 @@ org.apache.beam java-sdk-all - ${beam.version} com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-jackson2 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-protobuf - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - runtime com.google.apis google-api-services-dataflow - ${dataflow.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-clouddebugger - ${clouddebugger.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.cloud.bigdataoss util - 1.4.3 com.google.guava guava - - ${guava.version} com.google.guava guava-testlib - ${guava.version} test joda-time joda-time - ${joda.version} com.google.protobuf protobuf-java - ${protobuf.version} com.google.code.findbugs jsr305 - ${jsr305.version} com.fasterxml.jackson.core jackson-core - ${jackson.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} com.fasterxml.jackson.core jackson-databind - ${jackson.version} org.slf4j slf4j-api - ${slf4j.version} com.google.auto.service auto-service - 1.0-rc2 true com.google.auto.value auto-value - 1.1 provided @@ -513,35 +396,30 @@ org.hamcrest hamcrest-all - ${hamcrest.version} provided junit junit - ${junit.version} provided org.slf4j slf4j-jdk14 - ${slf4j.version} test org.mockito mockito-all - 1.10.19 test org.apache.beam java-sdk-all - ${beam.version} test-jar test @@ -549,7 +427,6 @@ com.google.cloud.dataflow google-cloud-dataflow-java-proto-library-all - 0.5.160304 test diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml deleted file mode 100644 index 033d8ae476de..000000000000 --- a/runners/spark/build-resources/checkstyle.xml +++ /dev/null @@ -1,225 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index eb0122994630..e6732464734a 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -25,6 +25,7 @@ org.apache.beam runners-parent 0.1.0-incubating-SNAPSHOT + ../pom.xml spark-runner @@ -35,9 +36,7 @@ UTF-8 UTF-8 - 1.7 1.6.1 - 0.1.0-incubating-SNAPSHOT @@ -68,18 +67,15 @@ com.google.guava guava - ${guava.version} com.google.auto.service auto-service - 1.0-rc2 true org.apache.beam java-sdk-all - ${beam.version} @@ -91,7 +87,6 @@ org.apache.beam java-examples-all - ${beam.version} @@ -118,13 +113,11 @@ junit junit - 4.12 test org.hamcrest hamcrest-all - 1.3 test @@ -132,155 +125,16 @@ - - org.apache.maven.plugins - maven-compiler-plugin - 3.5 - - ${java.version} - ${java.version} - - -Xlint:all,-serial - - - org.apache.maven.plugins maven-surefire-plugin + 2.19.1 1 false - - org.apache.maven.plugins - maven-checkstyle-plugin - - build-resources/checkstyle.xml - false - - - - org.apache.maven.plugins - maven-source-plugin - 2.4 - - - attach-sources - - jar-no-fork - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.3 - - ${java.version} - - - - attach-javadocs - - jar - - - - - - org.apache.maven.plugins - maven-clean-plugin - 3.0.0 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.8.2 - - - org.apache.maven.plugins - maven-install-plugin - 2.5.2 - - true - - - - org.apache.maven.plugins - maven-resources-plugin - 2.7 - - - org.apache.maven.plugins - maven-jar-plugin - 2.6 - - - org.apache.maven.plugins - maven-site-plugin - 3.4 - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - - org.apache.maven.scm - maven-scm-provider-gitexe - 1.9.2 - - - - - org.apache.maven.plugins - maven-gpg-plugin - 1.6 - - - sign-artifacts - verify - - sign - - - - - - org.apache.rat - apache-rat-plugin - 0.11 - - - .travis.yml - **/*.conf - **/*.iml - **/*.md - **/*.txt - **/.project - **/.checkstyle - **/.classpath - **/.settings/ - **/gen/** - **/resources/** - **/target/** - **/dependency-reduced-pom.xml - false - - - - - verify - verify - - check - - - - org.codehaus.mojo exec-maven-plugin @@ -321,7 +175,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.3 package @@ -352,12 +205,18 @@ org.apache.maven.plugins - maven-checkstyle-plugin + maven-compiler-plugin - org.apache.rat - apache-rat-plugin + org.apache.maven.plugins + maven-jar-plugin + + org.apache.maven.plugins maven-source-plugin diff --git a/sdks/java/src.xml b/sdks/java/build-tools/src/main/assembly/src.xml similarity index 100% rename from sdks/java/src.xml rename to sdks/java/build-tools/src/main/assembly/src.xml diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 03d224d71b60..a3b7cad0f61d 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -68,8 +68,6 @@ - org.apache.maven.plugins maven-checkstyle-plugin @@ -82,51 +80,20 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-javadoc-plugin - Google Cloud Dataflow SDK ${project.version} API - Google Cloud Dataflow SDK for Java, version ${project.version} + Apache Beam SDK ${project.version} API + Apache Beam SDK for Java, version ${project.version} ../javadoc/overview.html org.apache.beam.sdk @@ -195,7 +162,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 @@ -278,7 +244,6 @@ org.apache.avro avro-maven-plugin - ${avro.version} schemas @@ -301,7 +266,6 @@ org.codehaus.mojo build-helper-maven-plugin - 1.9.1 add-test-source @@ -324,7 +288,6 @@ io.grpc grpc-all - 0.12.0 @@ -364,69 +327,26 @@ com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-storage - ${storage.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - - - com.google.guava - guava-jdk5 - - - - org.codehaus.jackson - jackson-core-asl - - runtime com.google.http-client google-http-client-jackson2 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-protobuf - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - runtime com.google.oauth-client google-oauth-client-java6 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-datastore-protobuf - ${datastore.version} - - - - com.google.guava - guava-jdk5 - - - - com.google.http-client - google-http-client - - - com.google.api-client - google-api-client - - - com.google.oauth-client - google-oauth-client - - - com.google.http-client - google-http-client-jackson - - - com.google.http-client - google-http-client-protobuf - - com.google.cloud.bigdataoss gcsio - 1.4.3 com.google.cloud.bigdataoss util - 1.4.3 com.google.guava guava - - ${guava.version} com.google.guava guava-testlib - ${guava.version} test com.google.protobuf protobuf-java - ${protobuf.version} com.google.code.findbugs jsr305 - ${jsr305.version} com.fasterxml.jackson.core jackson-core - ${jackson.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} com.fasterxml.jackson.core jackson-databind - ${jackson.version} org.slf4j slf4j-api - ${slf4j.version} org.apache.avro avro - ${avro.version} @@ -626,7 +454,6 @@ joda-time joda-time - ${joda.version} org.hamcrest hamcrest-all - 1.3 test junit junit - 4.12 test diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index d1d5cd674895..18e46834b243 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -37,17 +37,36 @@ 0.2.3 + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + org.apache.beam java-sdk-all - ${project.version} io.grpc grpc-all - 0.12.0 @@ -66,7 +85,6 @@ org.apache.beam java-sdk-all - ${project.version} tests test @@ -74,21 +92,18 @@ org.hamcrest hamcrest-all - 1.3 test junit junit - 4.11 test org.slf4j slf4j-jdk14 - 1.7.14 test diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 3eaef628b556..e2d7eb2d1ec7 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -31,13 +31,32 @@ hdfs Apache Beam :: SDKs :: Java :: IO :: HDFS Library to read and write Hadoop/HDFS file formats from Beam. - jar + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.beam java-sdk-all - ${project.version} @@ -51,14 +70,12 @@ org.hamcrest hamcrest-all - 1.3 test junit junit - 4.11 test diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index ddfceed8ddb1..bb9baaa1f178 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -30,27 +30,32 @@ kafka Apache Beam :: SDKs :: Java :: IO :: Kafka Library to read Kafka topics. - jar - - - org.apache.maven.plugins - maven-compiler-plugin - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + org.apache.beam java-sdk-all - ${project.version} @@ -63,21 +68,18 @@ org.hamcrest hamcrest-all - ${hamcrest.version} test junit junit - ${junit.version} test org.slf4j slf4j-jdk14 - ${slf4j.version} test diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml index 27c40f7670b0..1032ae9245bf 100644 --- a/sdks/java/java8tests/pom.xml +++ b/sdks/java/java8tests/pom.xml @@ -35,8 +35,6 @@ This artifact includes tests of the SDK from a Java 8 user. - jar - @@ -77,42 +75,11 @@ org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -127,35 +94,30 @@ org.apache.beam java-sdk-all - ${project.version} test com.google.guava guava - ${guava.version} test joda-time joda-time - ${joda.version} test org.hamcrest hamcrest-all - ${hamcrest.version} test junit junit - ${junit.version} test From 376f88e9278650647f3c83c3265b3af6ed81acb4 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 3 May 2016 12:04:13 -0700 Subject: [PATCH 07/29] [BEAM-168] IntervalBEB: remove deprecated function The pre-commit wordcount test will confirm that this does not break the Cloud Dataflow worker. --- .../beam/sdk/util/IntervalBoundedExponentialBackOff.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java index a1d795b606a6..92ff2f08bc37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java @@ -54,12 +54,6 @@ public class IntervalBoundedExponentialBackOff implements BackOff { private final long initialIntervalMillis; private int currentAttempt; - // BEAM-168: https://issues.apache.org/jira/browse/BEAM-168 - @Deprecated - public IntervalBoundedExponentialBackOff(int maximumIntervalMillis, long initialIntervalMillis) { - this((long) maximumIntervalMillis, initialIntervalMillis); - } - public IntervalBoundedExponentialBackOff(long maximumIntervalMillis, long initialIntervalMillis) { Preconditions.checkArgument( maximumIntervalMillis > 0, "Maximum interval must be greater than zero."); From e6d48c0cd7eac93f14f7c4df3eee8a8936b9cc51 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 26 Apr 2016 19:55:34 -0700 Subject: [PATCH 08/29] Replace dataflow stagingLocation with tempLocation in example module. --- examples/java/README.md | 4 ++-- .../beam/examples/DebuggingWordCount.java | 2 +- .../beam/examples/MinimalWordCount.java | 12 ++++++------ .../beam/examples/WindowedWordCount.java | 2 +- .../org/apache/beam/examples/WordCount.java | 19 ++++++++----------- .../beam/examples/complete/AutoComplete.java | 4 ++-- .../apache/beam/examples/complete/TfIdf.java | 2 +- .../complete/TopWikipediaSessions.java | 2 +- .../examples/cookbook/BigQueryTornadoes.java | 2 +- .../cookbook/CombinePerKeyExamples.java | 2 +- .../examples/cookbook/DatastoreWordCount.java | 2 +- .../beam/examples/cookbook/DeDupExample.java | 12 +++++------- .../examples/cookbook/FilterExamples.java | 2 +- .../beam/examples/cookbook/JoinExamples.java | 2 +- .../examples/cookbook/MaxPerKeyExamples.java | 2 +- .../examples/complete/game/GameStats.java | 2 +- .../complete/game/HourlyTeamScore.java | 2 +- .../examples/complete/game/LeaderBoard.java | 2 +- .../examples/complete/game/UserScore.java | 2 +- 19 files changed, 37 insertions(+), 42 deletions(-) diff --git a/examples/java/README.md b/examples/java/README.md index cbcd01fc0f1c..4533303a23be 100644 --- a/examples/java/README.md +++ b/examples/java/README.md @@ -44,7 +44,7 @@ the same pipeline on fully managed resources in Google Cloud Platform: mvn compile exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ -Dexec.args="--project= \ - --stagingLocation= \ + --tempLocation= \ --runner=BlockingDataflowPipelineRunner" Make sure to use your project id, not the project number or the descriptive name. @@ -66,7 +66,7 @@ Platform: java -cp examples/target/google-cloud-dataflow-java-examples-all-bundled-.jar \ com.google.cloud.dataflow.examples.WordCount \ --project= \ - --stagingLocation= \ + --tempLocation= \ --runner=BlockingDataflowPipelineRunner Other examples can be run similarly by replacing the `WordCount` class path with the example classpath, e.g. diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 2e484651a634..11c56d132b0d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -68,7 +68,7 @@ * below, specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
index ec517c4bd69c..65d450dfce23 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -50,26 +50,26 @@
  *   4. Writing data to Cloud Storage as text files
  * 
* - *

To execute this pipeline, first edit the code to set your project ID, the staging + *

To execute this pipeline, first edit the code to set your project ID, the temp * location, and the output location. The specified GCS bucket(s) must already exist. * - *

Then, run the pipeline as described in the README. It will be deployed and run using the - * Dataflow service. No args are required to run the pipeline. You can see the results in your + *

Then, run the pipeline as described in the README. It will be deployed and run with the + * selected runner. No args are required to run the pipeline. You can see the results in your * output bucket in the GCS browser. */ public class MinimalWordCount { public static void main(String[] args) { - // Create a DataflowPipelineOptions object. This object lets us set various execution + // Create a PipelineOptions object. This object lets us set various execution // options for our pipeline, such as the associated Cloud Platform project and the location // in Google Cloud Storage to stage files. DataflowPipelineOptions options = PipelineOptionsFactory.create() - .as(DataflowPipelineOptions.class); + .as(DataflowPipelineOptions.class); options.setRunner(BlockingDataflowPipelineRunner.class); // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud. options.setProject("SET_YOUR_PROJECT_ID_HERE"); // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files. - options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY"); + options.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY"); // Create the Pipeline object with the options we defined above. Pipeline p = Pipeline.create(options); diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index ecb837e46eac..4d019bbf81c2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -83,7 +83,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 364d075c2b21..f5611d34146a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -37,9 +36,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - /** - * An example that counts words in Shakespeare and includes Dataflow best practices. + * An example that counts words in Shakespeare and includes Beam best practices. * *

This class, {@link WordCount}, is the second in a series of four successively more detailed * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}. @@ -56,13 +54,13 @@ * *

New Concepts: *

- *   1. Executing a Pipeline both locally and using the Dataflow service
+ *   1. Executing a Pipeline both locally and using the selected runner
  *   2. Using ParDo with static DoFns defined out-of-line
  *   3. Building a composite transform
  *   4. Defining your own pipeline options
  * 
* - *

Concept #1: you can execute this pipeline either locally or using the Dataflow service. + *

Concept #1: you can execute this pipeline either locally or using the selected runner. * These are now command-line options and not hard-coded as they were in the MinimalWordCount * example. * To execute this pipeline locally, specify general pipeline configuration: @@ -78,7 +76,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
@@ -173,17 +171,16 @@ public static interface WordCountOptions extends PipelineOptions { void setOutput(String value); /** - * Returns "gs://${YOUR_STAGING_DIRECTORY}/counts.txt" as the default destination. + * Returns "gs://${YOUR_TEMP_DIRECTORY}/counts.txt" as the default destination. */ public static class OutputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - if (dataflowOptions.getStagingLocation() != null) { - return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + if (options.getTempLocation() != null) { + return GcsPath.fromUri(options.getTempLocation()) .resolve("counts.txt").toString(); } else { - throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + throw new IllegalArgumentException("Must specify --output or --tempLocation"); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 2f2283cdb41f..c83a898aecdc 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -81,7 +81,7 @@ * specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=DataflowPipelineRunner
  *   --inputFile=gs://path/to/input*.txt
  * }
@@ -90,7 +90,7 @@ * specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=DataflowPipelineRunner
  *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
  *   --streaming
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
index 4b0db93d3e6e..1f1885fa0b6f 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -79,7 +79,7 @@
  * 

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * and an output prefix on GCS:
  *   --output=gs://YOUR_OUTPUT_PREFIX
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
index e7090fdf2f57..c483608ceb0e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -59,7 +59,7 @@
  * 

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index 47631aaf7274..80a2f25569ab 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -60,7 +60,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java index 1577b77bc279..505989c6fb84 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java @@ -67,7 +67,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://
+ *   --tempLocation=gs://
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 021fc14616b1..bfaecdf897ae 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -69,7 +69,7 @@ * }
* *

To run this example using Dataflow service, you must additionally - * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and + * provide either {@literal --tempLocation} or {@literal --tempLocation}, and * select one of the Dataflow pipeline runners, eg * {@literal --runner=BlockingDataflowPipelineRunner}. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index fe2bbc8573c3..37ddb4b8c29c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples.cookbook; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -43,7 +42,7 @@ * *

To execute this pipeline using the Dataflow service, specify pipeline configuration: * --project=YOUR_PROJECT_ID - * --stagingLocation=gs://YOUR_STAGING_DIRECTORY + * --tempLocation=gs://YOUR_TEMP_DIRECTORY * --runner=BlockingDataflowPipelineRunner * and an output prefix on GCS: * --output=gs://YOUR_OUTPUT_PREFIX @@ -69,16 +68,15 @@ private static interface Options extends PipelineOptions { String getOutput(); void setOutput(String value); - /** Returns gs://${STAGING_LOCATION}/"deduped.txt". */ + /** Returns gs://${TEMP_LOCATION}/"deduped.txt". */ public static class OutputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - if (dataflowOptions.getStagingLocation() != null) { - return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + if (options.getTempLocation() != null) { + return GcsPath.fromUri(options.getTempLocation()) .resolve("deduped.txt").toString(); } else { - throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + throw new IllegalArgumentException("Must specify --output or --tempLocation"); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java index 597ef701d434..1c6a6e1e9266 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java @@ -72,7 +72,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 7cf138f4352a..8e642f87fc7b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -55,7 +55,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java index 3bc120855b44..24809672e6dd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java @@ -61,7 +61,7 @@ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: *

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index 12dfdf9abebb..6187a1bb1e19 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -83,7 +83,7 @@ * like this: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index b516a322455b..845c56fa0336 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -59,7 +59,7 @@
  * like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  * }
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index a5d9fb93fb03..0b10f07f93ff 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -80,7 +80,7 @@
  * like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index 6a6c1cf03866..866adefdb886 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -62,7 +62,7 @@
  * the pipeline configuration like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  * }

From 86a68c2b3137b63a919d703d228b32ea4e1bbf4b Mon Sep 17 00:00:00 2001
From: Jason Kuster 
Date: Fri, 29 Apr 2016 17:25:23 -0700
Subject: [PATCH 09/29] Add Matcher serializer in TestPipeline.

---
 .../org/apache/beam/examples/WordCountIT.java |  18 +-
 .../testing/TestDataflowPipelineRunner.java   |  19 +-
 .../TestDataflowPipelineRunnerTest.java       | 220 ++++++++++++++++++
 .../beam/sdk/testing/MatcherDeserializer.java |  46 ++++
 .../beam/sdk/testing/MatcherSerializer.java   |  44 ++++
 .../beam/sdk/testing/SerializableMatcher.java |   7 +-
 .../apache/beam/sdk/testing/TestPipeline.java |  26 ++-
 .../beam/sdk/testing/TestPipelineOptions.java |  39 ++++
 .../beam/sdk/testing/TestPipelineTest.java    |  53 +++++
 9 files changed, 439 insertions(+), 33 deletions(-)
 create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java
 create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java

diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
index 56ca98c1a6b7..a09ec02881b0 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
@@ -18,13 +18,7 @@
 
 package org.apache.beam.examples;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
 import org.apache.beam.examples.WordCount.WordCountOptions;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineRunner;
-import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
@@ -34,6 +28,8 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
+import java.util.Date;
+
 /**
  * End-to-end tests of WordCount.
  */
@@ -43,8 +39,7 @@ public class WordCountIT {
   /**
    * Options for the WordCount Integration Test.
    */
-  public static interface WordCountITOptions extends TestPipelineOptions,
-         WordCountOptions, DataflowPipelineOptions {
+  public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions {
   }
 
   @Test
@@ -52,13 +47,8 @@ public void testE2EWordCount() throws Exception {
     PipelineOptionsFactory.register(WordCountITOptions.class);
     WordCountITOptions options = TestPipeline.testingPipelineOptions().as(WordCountITOptions.class);
     options.setOutput(Joiner.on("/").join(new String[]{options.getTempRoot(),
-        options.getJobName(), "output", "results"}));
+        String.format("WordCountIT-%tF-% {
   private static final String TENTATIVE_COUNTER = "tentative";
   private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
-  private static final Map EXECUTION_RESULTS =
-      new ConcurrentHashMap();
 
   private final TestDataflowPipelineOptions options;
   private final DataflowPipelineRunner runner;
@@ -87,10 +84,6 @@ public static TestDataflowPipelineRunner fromOptions(
     return new TestDataflowPipelineRunner(dataflowOptions);
   }
 
-  public static PipelineResult getPipelineResultByJobName(String jobName) {
-    return EXECUTION_RESULTS.get(jobName);
-  }
-
   @Override
   public DataflowPipelineJob run(Pipeline pipeline) {
     return run(pipeline, runner);
@@ -98,6 +91,7 @@ public DataflowPipelineJob run(Pipeline pipeline) {
 
   DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
 
+    TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class);
     final DataflowPipelineJob job;
     try {
       job = runner.run(pipeline);
@@ -108,6 +102,8 @@ DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
     LOG.info("Running Dataflow job {} with {} expected assertions.",
         job.getJobId(), expectedNumberOfAssertions);
 
+    assertThat(job, testPipelineOptions.getOnCreateMatcher());
+
     CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
         job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
 
@@ -151,6 +147,8 @@ public Optional call() throws Exception {
         throw new AssertionError(messageHandler.getErrorMessage() == null ?
             "The dataflow did not return a failure reason."
             : messageHandler.getErrorMessage());
+      } else {
+        assertThat(job, testPipelineOptions.getOnSuccessMatcher());
       }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
@@ -161,7 +159,6 @@ public Optional call() throws Exception {
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    EXECUTION_RESULTS.put(options.getJobName(), job);
     return job;
   }
 
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
index a45284c9afa9..fbaf116d98cb 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
@@ -19,6 +19,7 @@
 
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -36,10 +37,13 @@
 import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
 import org.apache.beam.runners.dataflow.util.TimeUtil;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.PipelineResult.State;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.SerializableMatcher;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.NoopPathValidator;
@@ -61,6 +65,8 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
 import org.joda.time.Instant;
 import org.junit.Before;
 import org.junit.Rule;
@@ -74,6 +80,7 @@
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
@@ -378,4 +385,217 @@ public State answer(InvocationOnMock invocation) {
     // instead of inside the try-catch block.
     fail("AssertionError expected");
   }
+
+  @Test
+  public void testBatchOnCreateMatcher() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testStreamingOnCreateMatcher() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.DONE);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.DONE);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestFailureMatcher());
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
+          any(JobMessagesHandler.class));
+      return;
+    }
+    fail("Expected an exception on pipeline failure.");
+  }
+
+  @Test
+  public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestFailureMatcher());
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.FAILED);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
+          any(JobMessagesHandler.class));
+      return;
+    }
+    fail("Expected an exception on pipeline failure.");
+  }
+
+  static class TestSuccessMatcher extends BaseMatcher implements
+      SerializableMatcher {
+    private final DataflowPipelineJob mockJob;
+    private final int called;
+
+    public TestSuccessMatcher(DataflowPipelineJob job, int times) {
+      this.mockJob = job;
+      this.called = times;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      if (!(o instanceof PipelineResult)) {
+        fail(String.format("Expected PipelineResult but received %s", o));
+      }
+      try {
+        verify(mockJob, Mockito.times(called)).waitToFinish(any(Long.class), any(TimeUnit.class),
+            any(JobMessagesHandler.class));
+      } catch (IOException | InterruptedException e) {
+        throw new AssertionError(e);
+      }
+      assertSame(mockJob, o);
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+    }
+  }
+
+  static class TestFailureMatcher extends BaseMatcher implements
+      SerializableMatcher {
+    @Override
+    public boolean matches(Object o) {
+      fail("OnSuccessMatcher should not be called on pipeline failure.");
+      return false;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+    }
+  }
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java
new file mode 100644
index 000000000000..84984709bc64
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.testing;
+
+import org.apache.beam.sdk.util.SerializableUtils;
+
+import com.google.api.client.util.Base64;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import java.io.IOException;
+
+/**
+ * MatcherDeserializer is used with Jackson to enable deserialization of SerializableMatchers.
+ */
+class MatcherDeserializer extends JsonDeserializer> {
+  @Override
+  public SerializableMatcher deserialize(JsonParser jsonParser,
+      DeserializationContext deserializationContext)
+      throws IOException, JsonProcessingException {
+    ObjectNode node = jsonParser.readValueAsTree();
+    String matcher = node.get("matcher").asText();
+    byte[] in = Base64.decodeBase64(matcher);
+    return (SerializableMatcher) SerializableUtils
+        .deserializeFromByteArray(in, "SerializableMatcher");
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java
new file mode 100644
index 000000000000..0feeae06ad05
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.testing;
+
+import com.google.api.client.util.Base64;
+
+import org.apache.beam.sdk.util.SerializableUtils;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+
+import java.io.IOException;
+
+/**
+ * MatcherSerializer is used with Jackson to enable serialization of SerializableMatchers.
+ */
+class MatcherSerializer extends JsonSerializer> {
+  @Override
+  public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator,
+      SerializerProvider serializerProvider) throws IOException, JsonProcessingException {
+    byte[] out = SerializableUtils.serializeToByteArray(matcher);
+    String encodedString = Base64.encodeBase64String(out);
+    jsonGenerator.writeStartObject();
+    jsonGenerator.writeStringField("matcher", encodedString);
+    jsonGenerator.writeEndObject();
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
index 9132db78726e..a465bbec32aa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.testing;
 
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import org.hamcrest.Matcher;
 
 import java.io.Serializable;
@@ -32,5 +34,8 @@
  *
  * @param  The type of value matched.
  */
-interface SerializableMatcher extends Matcher, Serializable {
+@JsonSerialize(using = MatcherSerializer.class)
+@JsonDeserialize(using = MatcherDeserializer.class)
+public interface SerializableMatcher extends Matcher, Serializable {
 }
+
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index a51a24e81270..a4921d56be0a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -31,7 +31,11 @@
 import com.google.common.base.Strings;
 import com.google.common.collect.Iterators;
 
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.TreeNode;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 import org.junit.experimental.categories.Category;
 
@@ -39,7 +43,7 @@
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.Map;
+import java.util.Map.Entry;
 
 import javax.annotation.Nullable;
 
@@ -157,16 +161,24 @@ public static PipelineOptions testingPipelineOptions() {
 
   public static String[] convertToArgs(PipelineOptions options) {
     try {
-      Map stringOpts = (Map) MAPPER.readValue(
-          MAPPER.writeValueAsBytes(options), Map.class).get("options");
+      byte[] opts = MAPPER.writeValueAsBytes(options);
 
+      JsonParser jsonParser = MAPPER.getFactory().createParser(opts);
+      TreeNode node = jsonParser.readValueAsTree();
+      ObjectNode optsNode = (ObjectNode) node.get("options");
       ArrayList optArrayList = new ArrayList<>();
-      for (Map.Entry entry : stringOpts.entrySet()) {
-        optArrayList.add("--" + entry.getKey() + "=" + entry.getValue());
+      Iterator> entries = optsNode.fields();
+      while (entries.hasNext()) {
+        Entry entry = entries.next();
+        if (entry.getValue().isTextual()) {
+          optArrayList.add("--" + entry.getKey() + "=" + entry.getValue().asText());
+        } else {
+          optArrayList.add("--" + entry.getKey() + "=" + entry.getValue());
+        }
       }
       return optArrayList.toArray(new String[optArrayList.size()]);
-    } catch (Exception e) {
-      return null;
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java
index 2599ae260d9f..ff553bafa85e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java
@@ -17,7 +17,12 @@
  */
 package org.apache.beam.sdk.testing;
 
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
 
 /**
  * {@link TestPipelineOptions} is a set of options for test pipelines.
@@ -27,4 +32,38 @@
 public interface TestPipelineOptions extends PipelineOptions {
   String getTempRoot();
   void setTempRoot(String value);
+
+  @Default.InstanceFactory(AlwaysPassMatcherFactory.class)
+  SerializableMatcher getOnCreateMatcher();
+  void setOnCreateMatcher(SerializableMatcher value);
+
+  @Default.InstanceFactory(AlwaysPassMatcherFactory.class)
+  SerializableMatcher getOnSuccessMatcher();
+  void setOnSuccessMatcher(SerializableMatcher value);
+
+  /**
+   * Factory for {@link PipelineResult} matchers which always pass.
+   */
+  class AlwaysPassMatcherFactory
+      implements DefaultValueFactory> {
+    @Override
+    public SerializableMatcher create(PipelineOptions options) {
+      return new AlwaysPassMatcher();
+    }
+  }
+
+  /**
+   * Matcher which will always pass.
+   */
+  class AlwaysPassMatcher extends BaseMatcher
+      implements SerializableMatcher {
+    @Override
+    public boolean matches(Object o) {
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+    }
+  }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
index 9460e13d2c84..8af4ff25bb06 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
@@ -23,6 +23,7 @@
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
+import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -31,6 +32,8 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
@@ -38,7 +41,9 @@
 import org.junit.runners.JUnit4;
 
 import java.util.Arrays;
+import java.util.Date;
 import java.util.List;
+import java.util.UUID;
 
 /** Tests for {@link TestPipeline}. */
 @RunWith(JUnit4.class)
@@ -116,4 +121,52 @@ public TestPipeline p() {
       return TestPipeline.create();
     }
   }
+
+  @Test
+  public void testMatcherSerializationDeserialization() {
+    TestPipelineOptions opts = PipelineOptionsFactory.as(TestPipelineOptions.class);
+    SerializableMatcher m1 = new TestMatcher();
+    SerializableMatcher m2 = new TestMatcher();
+
+    opts.setOnCreateMatcher(m1);
+    opts.setOnSuccessMatcher(m2);
+
+    String[] arr = TestPipeline.convertToArgs(opts);
+    TestPipelineOptions newOpts = PipelineOptionsFactory.fromArgs(arr)
+        .as(TestPipelineOptions.class);
+
+    assertEquals(m1, newOpts.getOnCreateMatcher());
+    assertEquals(m2, newOpts.getOnSuccessMatcher());
+  }
+
+  /**
+   * TestMatcher is a matcher designed for testing matcher serialization/deserialization.
+   */
+  public static class TestMatcher extends BaseMatcher
+      implements SerializableMatcher {
+    private final UUID uuid = UUID.randomUUID();
+    @Override
+    public boolean matches(Object o) {
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText(String.format("%tL", new Date()));
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof TestMatcher)) {
+        return false;
+      }
+      TestMatcher other = (TestMatcher) obj;
+      return other.uuid.equals(uuid);
+    }
+
+    @Override
+    public int hashCode() {
+      return uuid.hashCode();
+    }
+  }
 }

From 9ad04abc7cc49b415a1aa33a8baf7d058561f722 Mon Sep 17 00:00:00 2001
From: Luke Cwik 
Date: Tue, 3 May 2016 20:02:28 -0700
Subject: [PATCH 10/29] [BEAM-256] Address wrong import order and add millis to
 output path for WordCountIT

---
 .../src/test/java/org/apache/beam/examples/WordCountIT.java   | 2 +-
 .../java/org/apache/beam/sdk/testing/MatcherSerializer.java   | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
index a09ec02881b0..503445e6fbbd 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
@@ -47,7 +47,7 @@ public void testE2EWordCount() throws Exception {
     PipelineOptionsFactory.register(WordCountITOptions.class);
     WordCountITOptions options = TestPipeline.testingPipelineOptions().as(WordCountITOptions.class);
     options.setOutput(Joiner.on("/").join(new String[]{options.getTempRoot(),
-        String.format("WordCountIT-%tF-%
Date: Tue, 26 Apr 2016 18:41:37 -0700
Subject: [PATCH 11/29] [BEAM-53] Add PubsubApiaryClient, PubsubTestClient

* Move PubsubClient and friends out of sdk.io and into sdk.util.
* Add PubsubApiaryClient since gRPC has onerous boot class path
  requirements which I don't wish to inflict upon other runners.
* Add PubsubTestClient in preparation for unit testing
  PubsubUnbounded{Source,Sink}.
* Unit tests for all of above.
---
 .../org/apache/beam/sdk/io/PubsubClient.java  | 323 ------------
 .../java/org/apache/beam/sdk/io/PubsubIO.java | 252 ++++-----
 .../beam/sdk/util/PubsubApiaryClient.java     | 293 +++++++++++
 .../apache/beam/sdk/util/PubsubClient.java    | 489 ++++++++++++++++++
 .../sdk/{io => util}/PubsubGrpcClient.java    | 196 ++++---
 .../beam/sdk/util/PubsubTestClient.java       | 291 +++++++++++
 .../org/apache/beam/sdk/util/Transport.java   |   4 +
 .../org/apache/beam/sdk/io/PubsubIOTest.java  | 157 ------
 .../beam/sdk/util/PubsubApiaryClientTest.java | 134 +++++
 .../beam/sdk/util/PubsubClientTest.java       | 189 +++++++
 .../beam/sdk/util/PubsubGrpcClientTest.java   | 170 ++++++
 .../beam/sdk/util/PubsubTestClientTest.java   |  97 ++++
 12 files changed, 1885 insertions(+), 710 deletions(-)
 delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
 create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java
 create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java
 rename sdks/java/core/src/main/java/org/apache/beam/sdk/{io => util}/PubsubGrpcClient.java (68%)
 create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java
 create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java
 create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java
 create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java
 create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java

diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
deleted file mode 100644
index f92b480de1d8..000000000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * A helper interface for talking to Pubsub via an underlying transport.
- */
-public interface PubsubClient extends AutoCloseable {
-  /**
-   * Path representing a cloud project id.
-   */
-  class ProjectPath implements Serializable {
-    private final String path;
-
-    public ProjectPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      ProjectPath that = (ProjectPath) o;
-
-      return path.equals(that.path);
-
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static ProjectPath fromId(String projectId) {
-      return new ProjectPath(String.format("projects/%s", projectId));
-    }
-  }
-
-  /**
-   * Path representing a Pubsub subscription.
-   */
-  class SubscriptionPath implements Serializable {
-    private final String path;
-
-    public SubscriptionPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4);
-      return String.format("/subscriptions/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      SubscriptionPath that = (SubscriptionPath) o;
-      return path.equals(that.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static SubscriptionPath fromName(String projectId, String subscriptionName) {
-      return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
-          projectId, subscriptionName));
-    }
-  }
-
-  /**
-   * Path representing a Pubsub topic.
-   */
-  class TopicPath implements Serializable {
-    private final String path;
-
-    public TopicPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4);
-      return String.format("/topics/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      TopicPath topicPath = (TopicPath) o;
-      return path.equals(topicPath.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-
-    public static TopicPath fromName(String projectId, String topicName) {
-      return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
-    }
-  }
-
-  /**
-   * A message to be sent to Pubsub.
-   */
-  class OutgoingMessage {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    /**
-     * Timestamp for element (ms since epoch).
-     */
-    public final long timestampMsSinceEpoch;
-
-    public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) {
-      this.elementBytes = elementBytes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-    }
-  }
-
-  /**
-   * A message received from Pubsub.
-   */
-  class IncomingMessage {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    /**
-     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
-     * or the custom timestamp associated with the message.
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * Timestamp (in system time) at which we requested the message (ms since epoch).
-     */
-    public final long requestTimeMsSinceEpoch;
-
-    /**
-     * Id to pass back to Pubsub to acknowledge receipt of this message.
-     */
-    public final String ackId;
-
-    /**
-     * Id to pass to the runner to distinguish this message from all others.
-     */
-    public final byte[] recordId;
-
-    public IncomingMessage(
-        byte[] elementBytes,
-        long timestampMsSinceEpoch,
-        long requestTimeMsSinceEpoch,
-        String ackId,
-        byte[] recordId) {
-      this.elementBytes = elementBytes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
-      this.ackId = ackId;
-      this.recordId = recordId;
-    }
-  }
-
-  /**
-   * Gracefully close the underlying transport.
-   */
-  @Override
-  void close();
-
-
-  /**
-   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
-   * published.
-   *
-   * @throws IOException
-   */
-  int publish(TopicPath topic, Iterable outgoingMessages) throws IOException;
-
-  /**
-   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
-   * Return the received messages, or empty collection if none were available. Does not
-   * wait for messages to arrive. Returned messages will record heir request time
-   * as {@code requestTimeMsSinceEpoch}.
-   *
-   * @throws IOException
-   */
-  Collection pull(
-      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize)
-      throws IOException;
-
-  /**
-   * Acknowldege messages from {@code subscription} with {@code ackIds}.
-   *
-   * @throws IOException
-   */
-  void acknowledge(SubscriptionPath subscription, Iterable ackIds) throws IOException;
-
-  /**
-   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
-   * be {@code deadlineSeconds} from now.
-   *
-   * @throws IOException
-   */
-  void modifyAckDeadline(
-      SubscriptionPath subscription, Iterable ackIds,
-      int deadlineSeconds)
-      throws IOException;
-
-  /**
-   * Create {@code topic}.
-   *
-   * @throws IOException
-   */
-  void createTopic(TopicPath topic) throws IOException;
-
-  /*
-   * Delete {@code topic}.
-   *
-   * @throws IOException
-   */
-  void deleteTopic(TopicPath topic) throws IOException;
-
-  /**
-   * Return a list of topics for {@code project}.
-   *
-   * @throws IOException
-   */
-  Collection listTopics(ProjectPath project) throws IOException;
-
-  /**
-   * Create {@code subscription} to {@code topic}.
-   *
-   * @throws IOException
-   */
-  void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException;
-
-  /**
-   * Delete {@code subscription}.
-   *
-   * @throws IOException
-   */
-  void deleteSubscription(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return a list of subscriptions for {@code topic} in {@code project}.
-   *
-   * @throws IOException
-   */
-  Collection listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException;
-}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
index 46464614221a..fa867c24bd22 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
@@ -17,8 +17,7 @@
  */
 package org.apache.beam.sdk.io;
 
-import static com.google.common.base.MoreObjects.firstNonNull;
-import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -33,25 +32,18 @@
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.util.PubsubApiaryClient;
+import org.apache.beam.sdk.util.PubsubClient;
+import org.apache.beam.sdk.util.PubsubClient.IncomingMessage;
+import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage;
+import org.apache.beam.sdk.util.PubsubClient.TopicPath;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
 
-import com.google.api.client.util.Clock;
-import com.google.api.client.util.DateTime;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.AcknowledgeRequest;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.api.services.pubsub.model.PullRequest;
-import com.google.api.services.pubsub.model.PullResponse;
-import com.google.api.services.pubsub.model.ReceivedMessage;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
+import com.google.common.base.Strings;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -61,13 +53,10 @@
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Random;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import javax.annotation.Nullable;
 
 /**
@@ -82,6 +71,9 @@
 public class PubsubIO {
   private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class);
 
+  /** Factory for creating pubsub client to manage transport. */
+  private static final PubsubClient.PubsubClientFactory FACTORY = PubsubApiaryClient.FACTORY;
+
   /** The default {@link Coder} used to translate to/from Cloud Pub/Sub messages. */
   public static final Coder DEFAULT_PUBSUB_CODER = StringUtf8Coder.of();
 
@@ -142,48 +134,6 @@ private static void validatePubsubName(String name) {
     }
   }
 
-  /**
-   * Returns the {@link Instant} that corresponds to the timestamp in the supplied
-   * {@link PubsubMessage} under the specified {@code ink label}. See
-   * {@link PubsubIO.Read#timestampLabel(String)} for details about how these messages are
-   * parsed.
-   *
-   * 

The {@link Clock} parameter is used to virtualize time for testing. - * - * @throws IllegalArgumentException if the timestamp label is provided, but there is no - * corresponding attribute in the message or the value provided is not a valid timestamp - * string. - * @see PubsubIO.Read#timestampLabel(String) - */ - @VisibleForTesting - protected static Instant assignMessageTimestamp( - PubsubMessage message, @Nullable String label, Clock clock) { - if (label == null) { - return new Instant(clock.currentTimeMillis()); - } - - // Extract message attributes, defaulting to empty map if null. - Map attributes = firstNonNull( - message.getAttributes(), ImmutableMap.of()); - - String timestampStr = attributes.get(label); - checkArgument(timestampStr != null && !timestampStr.isEmpty(), - "PubSub message is missing a timestamp in label: %s", label); - - long millisSinceEpoch; - try { - // Try parsing as milliseconds since epoch. Note there is no way to parse a string in - // RFC 3339 format here. - // Expected IllegalArgumentException if parsing fails; we use that to fall back to RFC 3339. - millisSinceEpoch = Long.parseLong(timestampStr); - } catch (IllegalArgumentException e) { - // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an IllegalArgumentException - // if parsing fails, and the caller should handle. - millisSinceEpoch = DateTime.parseRfc3339(timestampStr).getValue(); - } - return new Instant(millisSinceEpoch); - } - /** * Class representing a Cloud Pub/Sub Subscription. */ @@ -679,8 +629,8 @@ public PCollection apply(PInput input) { if (boundedOutput) { return input.getPipeline().begin() - .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) - .apply(ParDo.of(new PubsubReader())).setCoder(coder); + .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo.of(new PubsubBoundedReader())).setCoder(coder); } else { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) @@ -740,86 +690,94 @@ public Duration getMaxReadTime() { return maxReadTime; } - private class PubsubReader extends DoFn { + /** + * Default reader when Pubsub subscription has some form of upper bound. + *

TODO: Consider replacing with BoundedReadFromUnboundedSource on top of upcoming + * PubsubUnboundedSource. + *

NOTE: This is not the implementation used when running on the Google Dataflow hosted + * service. + */ + private class PubsubBoundedReader extends DoFn { private static final int DEFAULT_PULL_SIZE = 100; + private static final int ACK_TIMEOUT_SEC = 60; @Override public void processElement(ProcessContext c) throws IOException { - Pubsub pubsubClient = - Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class)) - .build(); - - String subscription; - if (getSubscription() == null) { - String topic = getTopic().asPath(); - String[] split = topic.split("/"); - subscription = - "projects/" + split[1] + "/subscriptions/" + split[3] + "_dataflow_" - + new Random().nextLong(); - Subscription subInfo = new Subscription().setAckDeadlineSeconds(60).setTopic(topic); - try { - pubsubClient.projects().subscriptions().create(subscription, subInfo).execute(); - } catch (Exception e) { - throw new RuntimeException("Failed to create subscription: ", e); + try (PubsubClient pubsubClient = + FACTORY.newClient(timestampLabel, idLabel, + c.getPipelineOptions().as(PubsubOptions.class))) { + + PubsubClient.SubscriptionPath subscriptionPath; + if (getSubscription() == null) { + // Create a randomized subscription derived from the topic name. + String subscription = getTopic().topic + "_dataflow_" + new Random().nextLong(); + // The subscription will be registered under this pipeline's project if we know it. + // Otherwise we'll fall back to the topic's project. + // Note that they don't need to be the same. + String project = c.getPipelineOptions().as(PubsubOptions.class).getProject(); + if (Strings.isNullOrEmpty(project)) { + project = getTopic().project; + } + subscriptionPath = PubsubClient.subscriptionPathFromName(project, subscription); + TopicPath topicPath = + PubsubClient.topicPathFromName(getTopic().project, getTopic().topic); + try { + pubsubClient.createSubscription(topicPath, subscriptionPath, ACK_TIMEOUT_SEC); + } catch (Exception e) { + throw new RuntimeException("Failed to create subscription: ", e); + } + } else { + subscriptionPath = + PubsubClient.subscriptionPathFromName(getSubscription().project, + getSubscription().subscription); } - } else { - subscription = getSubscription().asPath(); - } - Instant endTime = (getMaxReadTime() == null) - ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime()); + Instant endTime = (getMaxReadTime() == null) + ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime()); - List messages = new ArrayList<>(); + List messages = new ArrayList<>(); - Throwable finallyBlockException = null; - try { - while ((getMaxNumRecords() == 0 || messages.size() < getMaxNumRecords()) - && Instant.now().isBefore(endTime)) { - PullRequest pullRequest = new PullRequest().setReturnImmediately(false); - if (getMaxNumRecords() > 0) { - pullRequest.setMaxMessages(getMaxNumRecords() - messages.size()); - } else { - pullRequest.setMaxMessages(DEFAULT_PULL_SIZE); - } + Throwable finallyBlockException = null; + try { + while ((getMaxNumRecords() == 0 || messages.size() < getMaxNumRecords()) + && Instant.now().isBefore(endTime)) { + int batchSize = DEFAULT_PULL_SIZE; + if (getMaxNumRecords() > 0) { + batchSize = Math.min(batchSize, getMaxNumRecords() - messages.size()); + } - PullResponse pullResponse = - pubsubClient.projects().subscriptions().pull(subscription, pullRequest).execute(); - List ackIds = new ArrayList<>(); - if (pullResponse.getReceivedMessages() != null) { - for (ReceivedMessage received : pullResponse.getReceivedMessages()) { - messages.add(received.getMessage()); - ackIds.add(received.getAckId()); + List batchMessages = + pubsubClient.pull(System.currentTimeMillis(), subscriptionPath, batchSize, + false); + List ackIds = new ArrayList<>(); + for (IncomingMessage message : batchMessages) { + messages.add(message); + ackIds.add(message.ackId); + } + if (ackIds.size() != 0) { + pubsubClient.acknowledge(subscriptionPath, ackIds); } } - - if (ackIds.size() != 0) { - AcknowledgeRequest ackRequest = new AcknowledgeRequest().setAckIds(ackIds); - pubsubClient.projects() - .subscriptions() - .acknowledge(subscription, ackRequest) - .execute(); + } catch (IOException e) { + throw new RuntimeException("Unexpected exception while reading from Pubsub: ", e); + } finally { + if (getSubscription() == null) { + try { + pubsubClient.deleteSubscription(subscriptionPath); + } catch (Exception e) { + finallyBlockException = e; + } } } - } catch (IOException e) { - throw new RuntimeException("Unexpected exception while reading from Pubsub: ", e); - } finally { - if (getTopic() != null) { - try { - pubsubClient.projects().subscriptions().delete(subscription).execute(); - } catch (IOException e) { - finallyBlockException = new RuntimeException("Failed to delete subscription: ", e); - LOG.error("Failed to delete subscription: ", e); - } + if (finallyBlockException != null) { + throw new RuntimeException("Failed to delete subscription: ", finallyBlockException); } - } - if (finallyBlockException != null) { - throw new RuntimeException(finallyBlockException); - } - for (PubsubMessage message : messages) { - c.outputWithTimestamp( - CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), - assignMessageTimestamp(message, getTimestampLabel(), Clock.SYSTEM)); + for (IncomingMessage message : messages) { + c.outputWithTimestamp( + CoderUtils.decodeFromByteArray(getCoder(), message.elementBytes), + new Instant(message.timestampMsSinceEpoch)); + } } } } @@ -1026,31 +984,28 @@ public Coder getCoder() { return coder; } + /** + * Writer to Pubsub which batches messages. + *

NOTE: This is not the implementation used when running on the Google Dataflow hosted + * service. + */ private class PubsubWriter extends DoFn { private static final int MAX_PUBLISH_BATCH_SIZE = 100; - private transient List output; - private transient Pubsub pubsubClient; + private transient List output; + private transient PubsubClient pubsubClient; @Override - public void startBundle(Context c) { + public void startBundle(Context c) throws IOException { this.output = new ArrayList<>(); - this.pubsubClient = - Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class)) - .build(); + this.pubsubClient = FACTORY.newClient(timestampLabel, idLabel, + c.getPipelineOptions().as(PubsubOptions.class)); } @Override public void processElement(ProcessContext c) throws IOException { - PubsubMessage message = - new PubsubMessage().encodeData(CoderUtils.encodeToByteArray(getCoder(), c.element())); - if (getTimestampLabel() != null) { - Map attributes = message.getAttributes(); - if (attributes == null) { - attributes = new HashMap<>(); - message.setAttributes(attributes); - } - attributes.put(getTimestampLabel(), String.valueOf(c.timestamp().getMillis())); - } + OutgoingMessage message = + new OutgoingMessage(CoderUtils.encodeToByteArray(getCoder(), c.element()), + c.timestamp().getMillis()); output.add(message); if (output.size() >= MAX_PUBLISH_BATCH_SIZE) { @@ -1063,13 +1018,16 @@ public void finishBundle(Context c) throws IOException { if (!output.isEmpty()) { publish(); } + output = null; + pubsubClient.close(); + pubsubClient = null; } private void publish() throws IOException { - PublishRequest publishRequest = new PublishRequest().setMessages(output); - pubsubClient.projects().topics() - .publish(getTopic().asPath(), publishRequest) - .execute(); + int n = pubsubClient.publish( + PubsubClient.topicPathFromName(getTopic().project, getTopic().topic), + output); + checkState(n == output.size()); output.clear(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java new file mode 100644 index 000000000000..f0a90968f510 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.Pubsub.Builder; +import com.google.api.services.pubsub.model.AcknowledgeRequest; +import com.google.api.services.pubsub.model.ListSubscriptionsResponse; +import com.google.api.services.pubsub.model.ListTopicsResponse; +import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PublishResponse; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.api.services.pubsub.model.PullRequest; +import com.google.api.services.pubsub.model.PullResponse; +import com.google.api.services.pubsub.model.ReceivedMessage; +import com.google.api.services.pubsub.model.Subscription; +import com.google.api.services.pubsub.model.Topic; +import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import javax.annotation.Nullable; + +/** + * A Pubsub client using Apiary. + */ +public class PubsubApiaryClient extends PubsubClient { + + public static final PubsubClientFactory FACTORY = new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + Pubsub pubsub = new Builder( + Transport.getTransport(), + Transport.getJsonFactory(), + new ChainingHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) + .setRootUrl(options.getPubsubRootUrl()) + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer(options.getGoogleApiTrace()) + .build(); + return new PubsubApiaryClient(timestampLabel, idLabel, pubsub); + } + }; + + /** + * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time + * instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids. + */ + @Nullable + private final String idLabel; + + /** + * Underlying Apiary client. + */ + private Pubsub pubsub; + + @VisibleForTesting + PubsubApiaryClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + Pubsub pubsub) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.pubsub = pubsub; + } + + @Override + public void close() { + // Nothing to close. + } + + @Override + public int publish(TopicPath topic, List outgoingMessages) + throws IOException { + List pubsubMessages = new ArrayList<>(outgoingMessages.size()); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes); + + Map attributes = pubsubMessage.getAttributes(); + if ((timestampLabel != null || idLabel != null) && attributes == null) { + attributes = new TreeMap<>(); + pubsubMessage.setAttributes(attributes); + } + + if (timestampLabel != null) { + attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + } + + if (idLabel != null) { + attributes.put(idLabel, + Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); + } + + pubsubMessages.add(pubsubMessage); + } + PublishRequest request = new PublishRequest().setMessages(pubsubMessages); + PublishResponse response = pubsub.projects() + .topics() + .publish(topic.getPath(), request) + .execute(); + return response.getMessageIds().size(); + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) throws IOException { + PullRequest request = new PullRequest() + .setReturnImmediately(returnImmediately) + .setMaxMessages(batchSize); + PullResponse response = pubsub.projects() + .subscriptions() + .pull(subscription.getPath(), request) + .execute(); + if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) { + return ImmutableList.of(); + } + List incomingMessages = new ArrayList<>(response.getReceivedMessages().size()); + for (ReceivedMessage message : response.getReceivedMessages()) { + PubsubMessage pubsubMessage = message.getMessage(); + @Nullable Map attributes = pubsubMessage.getAttributes(); + + // Payload. + byte[] elementBytes = pubsubMessage.decodeData(); + + // Timestamp. + long timestampMsSinceEpoch = + extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes); + + // Ack id. + String ackId = message.getAckId(); + checkState(!Strings.isNullOrEmpty(ackId)); + + // Record id, if any. + @Nullable byte[] recordId = null; + if (idLabel != null && attributes != null) { + String recordIdString = attributes.get(idLabel); + if (!Strings.isNullOrEmpty(recordIdString)) { + recordId = recordIdString.getBytes(); + } + } + if (recordId == null) { + recordId = pubsubMessage.getMessageId().getBytes(); + } + + incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, ackId, recordId)); + } + + return incomingMessages; + } + + @Override + public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { + AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds); + pubsub.projects() + .subscriptions() + .acknowledge(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) + throws IOException { + ModifyAckDeadlineRequest request = + new ModifyAckDeadlineRequest().setAckIds(ackIds) + .setAckDeadlineSeconds(deadlineSeconds); + pubsub.projects() + .subscriptions() + .modifyAckDeadline(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .create(topic.getPath(), new Topic()) + .execute(); // ignore Topic result. + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .delete(topic.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + ListTopicsResponse response = pubsub.projects() + .topics() + .list(project.getPath()) + .execute(); + if (response.getTopics() == null || response.getTopics().isEmpty()) { + return ImmutableList.of(); + } + List topics = new ArrayList<>(response.getTopics().size()); + for (Topic topic : response.getTopics()) { + topics.add(topicPathFromPath(topic.getName())); + } + return topics; + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, + int ackDeadlineSeconds) throws IOException { + Subscription request = new Subscription() + .setTopic(topic.getPath()) + .setAckDeadlineSeconds(ackDeadlineSeconds); + pubsub.projects() + .subscriptions() + .create(subscription.getPath(), request) + .execute(); // ignore Subscription result. + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + pubsub.projects() + .subscriptions() + .delete(subscription.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException { + ListSubscriptionsResponse response = pubsub.projects() + .subscriptions() + .list(project.getPath()) + .execute(); + if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) { + return ImmutableList.of(); + } + List subscriptions = new ArrayList<>(response.getSubscriptions().size()); + for (Subscription subscription : response.getSubscriptions()) { + if (subscription.getTopic().equals(topic.getPath())) { + subscriptions.add(subscriptionPathFromPath(subscription.getName())); + } + } + return subscriptions; + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute(); + return response.getAckDeadlineSeconds(); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java new file mode 100644 index 000000000000..a44329d9f1c6 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java @@ -0,0 +1,489 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.api.client.util.DateTime; +import com.google.common.base.Objects; +import com.google.common.base.Strings; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; + +/** + * An (abstract) helper class for talking to Pubsub via an underlying transport. + */ +public abstract class PubsubClient implements Closeable { + /** + * Factory for creating clients. + */ + public interface PubsubClientFactory extends Serializable { + /** + * Construct a new Pubsub client. It should be closed via {@link #close} in order + * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources + * construct). Uses {@code options} to derive pubsub endpoints and application credentials. + * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom + * timestamps/ids within message metadata. + */ + PubsubClient newClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + PubsubOptions options) throws IOException; + } + + /** + * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}. + * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException} + * if timestamp cannot be recognized. + */ + @Nullable + private static Long asMsSinceEpoch(@Nullable String timestamp) { + if (Strings.isNullOrEmpty(timestamp)) { + return null; + } + try { + // Try parsing as milliseconds since epoch. Note there is no way to parse a + // string in RFC 3339 format here. + // Expected IllegalArgumentException if parsing fails; we use that to fall back + // to RFC 3339. + return Long.parseLong(timestamp); + } catch (IllegalArgumentException e1) { + // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an + // IllegalArgumentException if parsing fails, and the caller should handle. + return DateTime.parseRfc3339(timestamp).getValue(); + } + } + + /** + * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code + * attributes} and {@code pubsubTimestamp}. + *

If {@code timestampLabel} is non-{@literal null} then the message attributes must contain + * that label, and the value of that label will be taken as the timestamp. + * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code + * pubsubTimestamp}. Throw {@link IllegalArgumentException} if the timestamp cannot be + * recognized as a ms-since-unix-epoch or RFC3339 time. + * + * @throws IllegalArgumentException + */ + protected static long extractTimestamp( + @Nullable String timestampLabel, + @Nullable String pubsubTimestamp, + @Nullable Map attributes) { + Long timestampMsSinceEpoch; + if (Strings.isNullOrEmpty(timestampLabel)) { + timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp); + checkArgument(timestampMsSinceEpoch != null, + "Cannot interpret PubSub publish timestamp: %s", + pubsubTimestamp); + } else { + String value = attributes == null ? null : attributes.get(timestampLabel); + checkArgument(value != null, + "PubSub message is missing a value for timestamp label %s", + timestampLabel); + timestampMsSinceEpoch = asMsSinceEpoch(value); + checkArgument(timestampMsSinceEpoch != null, + "Cannot interpret value of label %s as timestamp: %s", + timestampLabel, value); + } + return timestampMsSinceEpoch; + } + + /** + * Path representing a cloud project id. + */ + public static class ProjectPath implements Serializable { + private final String path; + + ProjectPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ProjectPath that = (ProjectPath) o; + + return path.equals(that.path); + + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static ProjectPath projectPathFromPath(String path) { + return new ProjectPath(path); + } + + public static ProjectPath projectPathFromId(String projectId) { + return new ProjectPath(String.format("projects/%s", projectId)); + } + + /** + * Path representing a Pubsub subscription. + */ + public static class SubscriptionPath implements Serializable { + private final String path; + + SubscriptionPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public String getV1Beta1Path() { + String[] splits = path.split("/"); + checkState(splits.length == 4, "Malformed subscription path %s", path); + return String.format("/subscriptions/%s/%s", splits[1], splits[3]); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SubscriptionPath that = (SubscriptionPath) o; + return path.equals(that.path); + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static SubscriptionPath subscriptionPathFromPath(String path) { + return new SubscriptionPath(path); + } + + public static SubscriptionPath subscriptionPathFromName( + String projectId, String subscriptionName) { + return new SubscriptionPath(String.format("projects/%s/subscriptions/%s", + projectId, subscriptionName)); + } + + /** + * Path representing a Pubsub topic. + */ + public static class TopicPath implements Serializable { + private final String path; + + TopicPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public String getV1Beta1Path() { + String[] splits = path.split("/"); + checkState(splits.length == 4, "Malformed topic path %s", path); + return String.format("/topics/%s/%s", splits[1], splits[3]); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TopicPath topicPath = (TopicPath) o; + return path.equals(topicPath.path); + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static TopicPath topicPathFromPath(String path) { + return new TopicPath(path); + } + + public static TopicPath topicPathFromName(String projectId, String topicName) { + return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName)); + } + + /** + * A message to be sent to Pubsub. + *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + public static class OutgoingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). + */ + public final long timestampMsSinceEpoch; + + public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + OutgoingMessage that = (OutgoingMessage) o; + + if (timestampMsSinceEpoch != that.timestampMsSinceEpoch) { + return false; + } + return Arrays.equals(elementBytes, that.elementBytes); + + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), timestampMsSinceEpoch); + } + } + + /** + * A message received from Pubsub. + *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + public static class IncomingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either Pubsub's processing time, + * or the custom timestamp associated with the message. + */ + public final long timestampMsSinceEpoch; + + /** + * Timestamp (in system time) at which we requested the message (ms since epoch). + */ + public final long requestTimeMsSinceEpoch; + + /** + * Id to pass back to Pubsub to acknowledge receipt of this message. + */ + public final String ackId; + + /** + * Id to pass to the runner to distinguish this message from all others. + */ + public final byte[] recordId; + + public IncomingMessage( + byte[] elementBytes, + long timestampMsSinceEpoch, + long requestTimeMsSinceEpoch, + String ackId, + byte[] recordId) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; + this.ackId = ackId; + this.recordId = recordId; + } + + public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) { + return new IncomingMessage(elementBytes, timestampMsSinceEpoch, requestTimeMsSinceEpoch, + ackId, recordId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IncomingMessage that = (IncomingMessage) o; + + if (timestampMsSinceEpoch != that.timestampMsSinceEpoch) { + return false; + } + if (requestTimeMsSinceEpoch != that.requestTimeMsSinceEpoch) { + return false; + } + if (!Arrays.equals(elementBytes, that.elementBytes)) { + return false; + } + if (!ackId.equals(that.ackId)) { + return false; + } + return Arrays.equals(recordId, that.recordId); + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), timestampMsSinceEpoch, + requestTimeMsSinceEpoch, + ackId, Arrays.hashCode(recordId)); + } + } + + /** + * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages + * published. + * + * @throws IOException + */ + public abstract int publish(TopicPath topic, List outgoingMessages) + throws IOException; + + /** + * Request the next batch of up to {@code batchSize} messages from {@code subscription}. + * Return the received messages, or empty collection if none were available. Does not + * wait for messages to arrive if {@code returnImmediately} is {@literal true}. + * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}. + * + * @throws IOException + */ + public abstract List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) + throws IOException; + + /** + * Acknowldege messages from {@code subscription} with {@code ackIds}. + * + * @throws IOException + */ + public abstract void acknowledge(SubscriptionPath subscription, List ackIds) + throws IOException; + + /** + * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to + * be {@code deadlineSeconds} from now. + * + * @throws IOException + */ + public abstract void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, + int deadlineSeconds) throws IOException; + + /** + * Create {@code topic}. + * + * @throws IOException + */ + public abstract void createTopic(TopicPath topic) throws IOException; + + /* + * Delete {@code topic}. + * + * @throws IOException + */ + public abstract void deleteTopic(TopicPath topic) throws IOException; + + /** + * Return a list of topics for {@code project}. + * + * @throws IOException + */ + public abstract List listTopics(ProjectPath project) throws IOException; + + /** + * Create {@code subscription} to {@code topic}. + * + * @throws IOException + */ + public abstract void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException; + + /** + * Delete {@code subscription}. + * + * @throws IOException + */ + public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException; + + /** + * Return a list of subscriptions for {@code topic} in {@code project}. + * + * @throws IOException + */ + public abstract List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException; + + /** + * Return the ack deadline, in seconds, for {@code subscription}. + * + * @throws IOException + */ + public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException; +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java similarity index 68% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java index 66fb61fb6335..b3c1b8f525bc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java @@ -16,13 +16,15 @@ * limitations under the License. */ -package org.apache.beam.sdk.io; +package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcpOptions; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; -import com.google.api.client.util.DateTime; import com.google.auth.oauth2.GoogleCredentials; -import com.google.common.base.Preconditions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; @@ -30,6 +32,7 @@ import com.google.pubsub.v1.AcknowledgeRequest; import com.google.pubsub.v1.DeleteSubscriptionRequest; import com.google.pubsub.v1.DeleteTopicRequest; +import com.google.pubsub.v1.GetSubscriptionRequest; import com.google.pubsub.v1.ListSubscriptionsRequest; import com.google.pubsub.v1.ListSubscriptionsResponse; import com.google.pubsub.v1.ListTopicsRequest; @@ -38,11 +41,13 @@ import com.google.pubsub.v1.PublishRequest; import com.google.pubsub.v1.PublishResponse; import com.google.pubsub.v1.PublisherGrpc; +import com.google.pubsub.v1.PublisherGrpc.PublisherBlockingStub; import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.PullRequest; import com.google.pubsub.v1.PullResponse; import com.google.pubsub.v1.ReceivedMessage; import com.google.pubsub.v1.SubscriberGrpc; +import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub; import com.google.pubsub.v1.Subscription; import com.google.pubsub.v1.Topic; @@ -56,7 +61,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -68,17 +72,44 @@ /** * A helper class for talking to Pubsub via grpc. */ -public class PubsubGrpcClient implements PubsubClient { +public class PubsubGrpcClient extends PubsubClient { private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com"; private static final int PUBSUB_PORT = 443; private static final List PUBSUB_SCOPES = Collections.singletonList("https://www.googleapis.com/auth/pubsub"); private static final int LIST_BATCH_SIZE = 1000; + private static final int DEFAULT_TIMEOUT_S = 15; + + public static final PubsubClientFactory FACTORY = + new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + ManagedChannel channel = NettyChannelBuilder + .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) + .negotiationType(NegotiationType.TLS) + .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) + .build(); + // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the + // various command line options. It currently only supports the older + // com.google.api.client.auth.oauth2.Credentials. + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + return new PubsubGrpcClient(timestampLabel, + idLabel, + DEFAULT_TIMEOUT_S, + channel, + credentials, + null /* publisher stub */, + null /* subscriber stub */); + } + }; + /** * Timeout for grpc calls (in s). */ - private static final int TIMEOUT_S = 15; + private final int timeoutSec; /** * Underlying netty channel, or {@literal null} if closed. @@ -104,6 +135,7 @@ public class PubsubGrpcClient implements PubsubClient { @Nullable private final String idLabel; + /** * Cached stubs, or null if not cached. */ @@ -111,35 +143,22 @@ public class PubsubGrpcClient implements PubsubClient { private PublisherGrpc.PublisherBlockingStub cachedPublisherStub; private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub; - private PubsubGrpcClient( - @Nullable String timestampLabel, @Nullable String idLabel, - ManagedChannel publisherChannel, GoogleCredentials credentials) { + @VisibleForTesting + PubsubGrpcClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + int timeoutSec, + ManagedChannel publisherChannel, + GoogleCredentials credentials, + PublisherGrpc.PublisherBlockingStub cachedPublisherStub, + SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub) { this.timestampLabel = timestampLabel; this.idLabel = idLabel; + this.timeoutSec = timeoutSec; this.publisherChannel = publisherChannel; this.credentials = credentials; - } - - /** - * Construct a new Pubsub grpc client. It should be closed via {@link #close} in order - * to ensure tidy cleanup of underlying netty resources. (Or use the try-with-resources - * construct since this class is {@link AutoCloseable}). If non-{@literal null}, use - * {@code timestampLabel} and {@code idLabel} to store custom timestamps/ids within - * message metadata. - */ - public static PubsubGrpcClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, - GcpOptions options) throws IOException { - ManagedChannel channel = NettyChannelBuilder - .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) - .negotiationType(NegotiationType.TLS) - .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) - .build(); - // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the - // various command line options. It currently only supports the older - // com.google.api.client.auth.oauth2.Credentials. - GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); - return new PubsubGrpcClient(timestampLabel, idLabel, channel, credentials); + this.cachedPublisherStub = cachedPublisherStub; + this.cachedSubscriberStub = cachedSubscriberStub; } /** @@ -147,24 +166,34 @@ public static PubsubGrpcClient newClient( */ @Override public void close() { - Preconditions.checkState(publisherChannel != null, "Client has already been closed"); - publisherChannel.shutdown(); - try { - publisherChannel.awaitTermination(TIMEOUT_S, TimeUnit.SECONDS); - } catch (InterruptedException e) { - // Ignore. - Thread.currentThread().interrupt(); + if (publisherChannel == null) { + // Already closed. + return; } - publisherChannel = null; + // Can gc the underlying stubs. cachedPublisherStub = null; cachedSubscriberStub = null; + // Mark the client as having been closed before going further + // in case we have an exception from the channel. + ManagedChannel publisherChannel = this.publisherChannel; + this.publisherChannel = null; + // Gracefully shutdown the channel. + publisherChannel.shutdown(); + if (timeoutSec > 0) { + try { + publisherChannel.awaitTermination(timeoutSec, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // Ignore. + Thread.currentThread().interrupt(); + } + } } /** * Return channel with interceptor for returning credentials. */ private Channel newChannel() throws IOException { - Preconditions.checkState(publisherChannel != null, "PubsubGrpcClient has been closed"); + checkState(publisherChannel != null, "PubsubGrpcClient has been closed"); ClientAuthInterceptor interceptor = new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor()); return ClientInterceptors.intercept(publisherChannel, interceptor); @@ -173,25 +202,33 @@ private Channel newChannel() throws IOException { /** * Return a stub for making a publish request with a timeout. */ - private PublisherGrpc.PublisherBlockingStub publisherStub() throws IOException { + private PublisherBlockingStub publisherStub() throws IOException { if (cachedPublisherStub == null) { cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel()); } - return cachedPublisherStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + if (timeoutSec > 0) { + return cachedPublisherStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); + } else { + return cachedPublisherStub; + } } /** * Return a stub for making a subscribe request with a timeout. */ - private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOException { + private SubscriberBlockingStub subscriberStub() throws IOException { if (cachedSubscriberStub == null) { cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel()); } - return cachedSubscriberStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + if (timeoutSec > 0) { + return cachedSubscriberStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); + } else { + return cachedSubscriberStub; + } } @Override - public int publish(TopicPath topic, Iterable outgoingMessages) + public int publish(TopicPath topic, List outgoingMessages) throws IOException { PublishRequest.Builder request = PublishRequest.newBuilder() .setTopic(topic.getPath()); @@ -208,7 +245,7 @@ public int publish(TopicPath topic, Iterable outgoingMessages) if (idLabel != null) { message.getMutableAttributes() .put(idLabel, - Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); + Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); } request.addMessages(message); @@ -219,13 +256,14 @@ public int publish(TopicPath topic, Iterable outgoingMessages) } @Override - public Collection pull( + public List pull( long requestTimeMsSinceEpoch, SubscriptionPath subscription, - int batchSize) throws IOException { + int batchSize, + boolean returnImmediately) throws IOException { PullRequest request = PullRequest.newBuilder() .setSubscription(subscription.getPath()) - .setReturnImmediately(true) + .setReturnImmediately(returnImmediately) .setMaxMessages(batchSize) .build(); PullResponse response = subscriberStub().pull(request); @@ -235,41 +273,24 @@ public Collection pull( List incomingMessages = new ArrayList<>(response.getReceivedMessagesCount()); for (ReceivedMessage message : response.getReceivedMessagesList()) { PubsubMessage pubsubMessage = message.getMessage(); - Map attributes = pubsubMessage.getAttributes(); + @Nullable Map attributes = pubsubMessage.getAttributes(); // Payload. byte[] elementBytes = pubsubMessage.getData().toByteArray(); // Timestamp. - // Start with Pubsub processing time. + String pubsubTimestampString = null; Timestamp timestampProto = pubsubMessage.getPublishTime(); - long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L; - if (timestampLabel != null && attributes != null) { - String timestampString = attributes.get(timestampLabel); - if (timestampString != null && !timestampString.isEmpty()) { - try { - // Try parsing as milliseconds since epoch. Note there is no way to parse a - // string in RFC 3339 format here. - // Expected IllegalArgumentException if parsing fails; we use that to fall back - // to RFC 3339. - timestampMsSinceEpoch = Long.parseLong(timestampString); - } catch (IllegalArgumentException e1) { - try { - // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an - // IllegalArgumentException if parsing fails, and the caller should handle. - timestampMsSinceEpoch = DateTime.parseRfc3339(timestampString).getValue(); - } catch (IllegalArgumentException e2) { - // Fallback to Pubsub processing time. - } - } - } - // else: fallback to Pubsub processing time. + if (timestampProto != null) { + pubsubTimestampString = String.valueOf(timestampProto.getSeconds() + + timestampProto.getNanos() / 1000L); } - // else: fallback to Pubsub processing time. + long timestampMsSinceEpoch = + extractTimestamp(timestampLabel, pubsubTimestampString, attributes); // Ack id. String ackId = message.getAckId(); - Preconditions.checkState(ackId != null && !ackId.isEmpty()); + checkState(!Strings.isNullOrEmpty(ackId)); // Record id, if any. @Nullable byte[] recordId = null; @@ -284,13 +305,13 @@ public Collection pull( } incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch, - requestTimeMsSinceEpoch, ackId, recordId)); + requestTimeMsSinceEpoch, ackId, recordId)); } return incomingMessages; } @Override - public void acknowledge(SubscriptionPath subscription, Iterable ackIds) + public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { AcknowledgeRequest request = AcknowledgeRequest.newBuilder() .setSubscription(subscription.getPath()) @@ -301,8 +322,7 @@ public void acknowledge(SubscriptionPath subscription, Iterable ackIds) @Override public void modifyAckDeadline( - SubscriptionPath subscription, Iterable ackIds, int - deadlineSeconds) + SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { ModifyAckDeadlineRequest request = ModifyAckDeadlineRequest.newBuilder() @@ -330,7 +350,7 @@ public void deleteTopic(TopicPath topic) throws IOException { } @Override - public Collection listTopics(ProjectPath project) throws IOException { + public List listTopics(ProjectPath project) throws IOException { ListTopicsRequest.Builder request = ListTopicsRequest.newBuilder() .setProject(project.getPath()) @@ -342,7 +362,7 @@ public Collection listTopics(ProjectPath project) throws IOException List topics = new ArrayList<>(response.getTopicsCount()); while (true) { for (Topic topic : response.getTopicsList()) { - topics.add(new TopicPath(topic.getName())); + topics.add(topicPathFromPath(topic.getName())); } if (response.getNextPageToken().isEmpty()) { break; @@ -375,7 +395,7 @@ public void deleteSubscription(SubscriptionPath subscription) throws IOException } @Override - public Collection listSubscriptions(ProjectPath project, TopicPath topic) + public List listSubscriptions(ProjectPath project, TopicPath topic) throws IOException { ListSubscriptionsRequest.Builder request = ListSubscriptionsRequest.newBuilder() @@ -389,7 +409,7 @@ public Collection listSubscriptions(ProjectPath project, Topic while (true) { for (Subscription subscription : response.getSubscriptionsList()) { if (subscription.getTopic().equals(topic.getPath())) { - subscriptions.add(new SubscriptionPath(subscription.getName())); + subscriptions.add(subscriptionPathFromPath(subscription.getName())); } } if (response.getNextPageToken().isEmpty()) { @@ -400,4 +420,14 @@ public Collection listSubscriptions(ProjectPath project, Topic } return subscriptions; } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + GetSubscriptionRequest request = + GetSubscriptionRequest.newBuilder() + .setSubscription(subscription.getPath()) + .build(); + Subscription response = subscriberStub().getSubscription(request); + return response.getAckDeadlineSeconds(); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java new file mode 100644 index 000000000000..4a47c3070776 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; + +/** + * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for + * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline} + * methods. + */ +public class PubsubTestClient extends PubsubClient { + public static PubsubClientFactory createFactoryForPublish( + final TopicPath expectedTopic, + final Set expectedOutgoingMessages) { + return new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(expectedTopic, null, 0, expectedOutgoingMessages, null); + } + }; + } + + public static PubsubClientFactory createFactoryForPull( + @Nullable final SubscriptionPath expectedSubscription, + final int ackTimeoutSec, + @Nullable final List expectedIncomingMessages) { + return new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(null, expectedSubscription, ackTimeoutSec, + null, expectedIncomingMessages); + } + }; + } + + /** + * Only publish calls for this topic are allowed. + */ + @Nullable + private TopicPath expectedTopic; + /** + * Only pull calls for this subscription are allowed. + */ + @Nullable + private SubscriptionPath expectedSubscription; + + /** + * Timeout to simulate. + */ + private int ackTimeoutSec; + + /** + * Messages yet to seen in a {@link #publish} call. + */ + @Nullable + private Set remainingExpectedOutgoingMessages; + + /** + * Messages waiting to be received by a {@link #pull} call. + */ + @Nullable + private List remainingPendingIncomingMessages; + + /** + * Messages which have been returned from a {@link #pull} call and + * not yet ACKed by an {@link #acknowledge} call. + */ + private Map pendingAckIncommingMessages; + + /** + * When above messages are due to have their ACK deadlines expire. + */ + private Map ackDeadline; + + /** + * Current time. + */ + private long nowMsSinceEpoch; + + @VisibleForTesting + PubsubTestClient( + @Nullable TopicPath expectedTopic, + @Nullable SubscriptionPath expectedSubscription, + int ackTimeoutSec, + @Nullable Set expectedOutgoingMessages, + @Nullable List expectedIncomingMessages) { + this.expectedTopic = expectedTopic; + this.expectedSubscription = expectedSubscription; + this.ackTimeoutSec = ackTimeoutSec; + + this.remainingExpectedOutgoingMessages = expectedOutgoingMessages; + this.remainingPendingIncomingMessages = expectedIncomingMessages; + + this.pendingAckIncommingMessages = new HashMap<>(); + this.ackDeadline = new HashMap<>(); + this.nowMsSinceEpoch = Long.MIN_VALUE; + } + + /** + * Advance wall-clock time to {@code newNowMsSinceEpoch}. This will simulate Pubsub expiring + * outstanding ACKs. + */ + public void advanceTo(long newNowMsSinceEpoch) { + checkArgument(newNowMsSinceEpoch >= nowMsSinceEpoch, + "Cannot advance time backwards from %d to %d", nowMsSinceEpoch, + newNowMsSinceEpoch); + nowMsSinceEpoch = newNowMsSinceEpoch; + // Any messages who's ACKs timed out are available for re-pulling. + Iterator> deadlineItr = ackDeadline.entrySet().iterator(); + while (deadlineItr.hasNext()) { + Map.Entry entry = deadlineItr.next(); + if (entry.getValue() <= nowMsSinceEpoch) { + remainingPendingIncomingMessages.add(pendingAckIncommingMessages.remove(entry.getKey())); + deadlineItr.remove(); + } + } + } + + @Override + public void close() { + if (remainingExpectedOutgoingMessages != null) { + checkState(this.remainingExpectedOutgoingMessages.isEmpty(), + "Failed to pull %d messages", this.remainingExpectedOutgoingMessages.size()); + remainingExpectedOutgoingMessages = null; + } + if (remainingPendingIncomingMessages != null) { + checkState(remainingPendingIncomingMessages.isEmpty(), + "Failed to publish %d messages", remainingPendingIncomingMessages.size()); + checkState(pendingAckIncommingMessages.isEmpty(), + "Failed to ACK %d messages", pendingAckIncommingMessages.size()); + checkState(ackDeadline.isEmpty(), + "Failed to ACK %d messages", ackDeadline.size()); + remainingPendingIncomingMessages = null; + pendingAckIncommingMessages = null; + ackDeadline = null; + } + } + + @Override + public int publish( + TopicPath topic, List outgoingMessages) throws IOException { + checkNotNull(expectedTopic, "Missing expected topic"); + checkNotNull(remainingExpectedOutgoingMessages, "Missing expected outgoing messages"); + checkState(topic.equals(expectedTopic), "Topic %s does not match expected %s", topic, + expectedTopic); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + checkState(remainingExpectedOutgoingMessages.remove(outgoingMessage), + "Unexpeced outgoing message %s", outgoingMessage); + } + return outgoingMessages.size(); + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize, + boolean returnImmediately) throws IOException { + checkState(requestTimeMsSinceEpoch == nowMsSinceEpoch, + "Simulated time %d does not match requset time %d", nowMsSinceEpoch, + requestTimeMsSinceEpoch); + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + checkState(returnImmediately, "PubsubTestClient only supports returning immediately"); + + List incomingMessages = new ArrayList<>(); + Iterator pendItr = remainingPendingIncomingMessages.iterator(); + while (pendItr.hasNext()) { + IncomingMessage incomingMessage = pendItr.next(); + pendItr.remove(); + IncomingMessage incomingMessageWithRequestTime = + incomingMessage.withRequestTime(requestTimeMsSinceEpoch); + incomingMessages.add(incomingMessageWithRequestTime); + pendingAckIncommingMessages.put(incomingMessageWithRequestTime.ackId, + incomingMessageWithRequestTime); + ackDeadline.put(incomingMessageWithRequestTime.ackId, + requestTimeMsSinceEpoch + ackTimeoutSec * 1000); + if (incomingMessages.size() >= batchSize) { + break; + } + } + return incomingMessages; + } + + @Override + public void acknowledge( + SubscriptionPath subscription, + List ackIds) throws IOException { + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + + for (String ackId : ackIds) { + checkState(ackDeadline.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + checkState(pendingAckIncommingMessages.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + } + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + + for (String ackId : ackIds) { + checkState(ackDeadline.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + checkState(pendingAckIncommingMessages.containsKey(ackId), + "No message with ACK id %s is outstanding", ackId); + ackDeadline.put(ackId, nowMsSinceEpoch + deadlineSeconds * 1000); + } + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listSubscriptions( + ProjectPath project, TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + return ackTimeoutSec; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java index 9082ce375b7f..6daecdb6ce60 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java @@ -105,7 +105,11 @@ private static ApiComponents apiComponentsFromUrl(String urlString) { /** * Returns a Pubsub client builder using the specified {@link PubsubOptions}. + * + * @deprecated Use an appropriate + * {@link org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory} */ + @Deprecated public static Pubsub.Builder newPubsubClient(PubsubOptions options) { return new Pubsub.Builder(getTransport(), getJsonFactory(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java index 1e5bf5157b05..eaf452d8bafa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java @@ -24,22 +24,13 @@ import org.apache.beam.sdk.transforms.display.DisplayData; -import com.google.api.client.testing.http.FixedClock; -import com.google.api.client.util.Clock; -import com.google.api.services.pubsub.model.PubsubMessage; - import org.joda.time.Duration; -import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashMap; - -import javax.annotation.Nullable; - /** * Tests for PubsubIO Read and Write transforms. */ @@ -90,154 +81,6 @@ public void testTopicValidationTooLong() throws Exception { .toString()); } - /** - * Helper function that creates a {@link PubsubMessage} with the given timestamp registered as - * an attribute with the specified label. - * - *

If {@code label} is {@code null}, then the attributes are {@code null}. - * - *

Else, if {@code timestamp} is {@code null}, then attributes are present but have no key for - * the label. - */ - private static PubsubMessage messageWithTimestamp( - @Nullable String label, @Nullable String timestamp) { - PubsubMessage message = new PubsubMessage(); - if (label == null) { - message.setAttributes(null); - return message; - } - - message.setAttributes(new HashMap()); - - if (timestamp == null) { - return message; - } - - message.getAttributes().put(label, timestamp); - return message; - } - - /** - * Helper function that parses the given string to a timestamp through the PubSubIO plumbing. - */ - private static Instant parseTimestamp(@Nullable String timestamp) { - PubsubMessage message = messageWithTimestamp("mylabel", timestamp); - return PubsubIO.assignMessageTimestamp(message, "mylabel", Clock.SYSTEM); - } - - @Test - public void noTimestampLabelReturnsNow() { - final long time = 987654321L; - Instant timestamp = PubsubIO.assignMessageTimestamp( - messageWithTimestamp(null, null), null, new FixedClock(time)); - - assertEquals(new Instant(time), timestamp); - } - - @Test - public void timestampLabelWithNullAttributesThrowsError() { - PubsubMessage message = messageWithTimestamp(null, null); - thrown.expect(RuntimeException.class); - thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); - - PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); - } - - @Test - public void timestampLabelSetWithMissingAttributeThrowsError() { - PubsubMessage message = messageWithTimestamp("notMyLabel", "ignored"); - thrown.expect(RuntimeException.class); - thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); - - PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); - } - - @Test - public void timestampLabelParsesMillisecondsSinceEpoch() { - Long millis = 1446162101123L; - assertEquals(new Instant(millis), parseTimestamp(millis.toString())); - } - - @Test - public void timestampLabelParsesRfc3339Seconds() { - String rfc3339 = "2015-10-29T23:41:41Z"; - assertEquals(Instant.parse(rfc3339), parseTimestamp(rfc3339)); - } - - @Test - public void timestampLabelParsesRfc3339Tenths() { - String rfc3339tenths = "2015-10-29T23:41:41.1Z"; - assertEquals(Instant.parse(rfc3339tenths), parseTimestamp(rfc3339tenths)); - } - - @Test - public void timestampLabelParsesRfc3339Hundredths() { - String rfc3339hundredths = "2015-10-29T23:41:41.12Z"; - assertEquals(Instant.parse(rfc3339hundredths), parseTimestamp(rfc3339hundredths)); - } - - @Test - public void timestampLabelParsesRfc3339Millis() { - String rfc3339millis = "2015-10-29T23:41:41.123Z"; - assertEquals(Instant.parse(rfc3339millis), parseTimestamp(rfc3339millis)); - } - - @Test - public void timestampLabelParsesRfc3339Micros() { - String rfc3339micros = "2015-10-29T23:41:41.123456Z"; - assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); - // Note: micros part 456/1000 is dropped. - assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); - } - - @Test - public void timestampLabelParsesRfc3339MicrosRounding() { - String rfc3339micros = "2015-10-29T23:41:41.123999Z"; - assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); - // Note: micros part 999/1000 is dropped, not rounded up. - assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); - } - - @Test - public void timestampLabelWithInvalidFormatThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("not-a-timestamp"); - } - - @Test - public void timestampLabelWithInvalidFormat2ThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("null"); - } - - @Test - public void timestampLabelWithInvalidFormat3ThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("2015-10"); - } - - @Test - public void timestampLabelParsesRfc3339WithSmallYear() { - // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted - // This is therefore a "small year" until this difference is reconciled. - String rfc3339SmallYear = "1582-10-15T01:23:45.123Z"; - assertEquals(Instant.parse(rfc3339SmallYear), parseTimestamp(rfc3339SmallYear)); - } - - @Test - public void timestampLabelParsesRfc3339WithLargeYear() { - // Year 9999 in range. - String rfc3339LargeYear = "9999-10-29T23:41:41.123999Z"; - assertEquals(Instant.parse(rfc3339LargeYear), parseTimestamp(rfc3339LargeYear)); - } - - @Test - public void timestampLabelRfc3339WithTooLargeYearThrowsError() { - thrown.expect(NumberFormatException.class); - // Year 10000 out of range. - parseTimestamp("10000-10-29T23:41:41.123999Z"); - } - @Test public void testReadDisplayData() { String topic = "projects/project/topics/topic"; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java new file mode 100644 index 000000000000..40c31fb5ac03 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PublishResponse; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.api.services.pubsub.model.PullRequest; +import com.google.api.services.pubsub.model.PullResponse; +import com.google.api.services.pubsub.model.ReceivedMessage; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.Hashing; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubApiaryClient. + */ +public class PubsubApiaryClientTest { + private Pubsub mockPubsub; + private PubsubClient client; + + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long PUB_TIME = 3456L; + private static final long MESSAGE_TIME = 6789L; + private static final String TIMESTAMP_LABEL = "timestamp"; + private static final String ID_LABEL = "id"; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String CUSTOM_ID = + Hashing.murmur3_128().hashBytes(DATA.getBytes()).toString(); + private static final String ACK_ID = "testAckId"; + + @Before + public void setup() throws IOException { + mockPubsub = Mockito.mock(Pubsub.class, Mockito.RETURNS_DEEP_STUBS); + client = new PubsubApiaryClient(TIMESTAMP_LABEL, ID_LABEL, mockPubsub); + } + + @After + public void teardown() throws IOException { + client.close(); + client = null; + mockPubsub = null; + } + + @Test + public void pullOneMessage() throws IOException { + String expectedSubscription = SUBSCRIPTION.getPath(); + PullRequest expectedRequest = + new PullRequest().setReturnImmediately(true).setMaxMessages(10); + PubsubMessage expectedPubsubMessage = new PubsubMessage() + .setMessageId(MESSAGE_ID) + .encodeData(DATA.getBytes()) + .setPublishTime(String.valueOf(PUB_TIME)) + .setAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)); + ReceivedMessage expectedReceivedMessage = + new ReceivedMessage().setMessage(expectedPubsubMessage) + .setAckId(ACK_ID); + PullResponse expectedResponse = + new PullResponse().setReceivedMessages(ImmutableList.of(expectedReceivedMessage)); + Mockito.when(mockPubsub.projects() + .subscriptions() + .pull(expectedSubscription, expectedRequest) + .execute()) + .thenReturn(expectedResponse); + List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); + assertEquals(1, acutalMessages.size()); + IncomingMessage actualMessage = acutalMessages.get(0); + assertEquals(ACK_ID, actualMessage.ackId); + assertEquals(DATA, new String(actualMessage.elementBytes)); + assertEquals(CUSTOM_ID, new String(actualMessage.recordId)); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + } + + @Test + public void publishOneMessage() throws IOException { + String expectedTopic = TOPIC.getPath(); + PubsubMessage expectedPubsubMessage = new PubsubMessage() + .encodeData(DATA.getBytes()) + .setAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)); + PublishRequest expectedRequest = new PublishRequest() + .setMessages(ImmutableList.of(expectedPubsubMessage)); + PublishResponse expectedResponse = new PublishResponse() + .setMessageIds(ImmutableList.of(MESSAGE_ID)); + Mockito.when(mockPubsub.projects() + .topics() + .publish(expectedTopic, expectedRequest) + .execute()) + .thenReturn(expectedResponse); + OutgoingMessage actualMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); + assertEquals(1, n); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java new file mode 100644 index 000000000000..22508572e5a0 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.ProjectPath; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.common.collect.ImmutableMap; + +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Map; + +/** + * Tests for helper classes and methods in PubsubClient. + */ +public class PubsubClientTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + // + // Timestamp handling + // + + private long parse(String timestamp) { + Map map = ImmutableMap.of("myLabel", timestamp); + return PubsubClient.extractTimestamp("myLabel", null, map); + } + + private void roundTripRfc339(String timestamp) { + assertEquals(Instant.parse(timestamp).getMillis(), parse(timestamp)); + } + + private void truncatedRfc339(String timestamp, String truncatedTimestmap) { + assertEquals(Instant.parse(truncatedTimestmap).getMillis(), parse(timestamp)); + } + + @Test + public void noTimestampLabelReturnsPubsubPublish() { + final long time = 987654321L; + long timestamp = PubsubClient.extractTimestamp(null, String.valueOf(time), null); + assertEquals(time, timestamp); + } + + @Test + public void noTimestampLabelAndInvalidPubsubPublishThrowsError() { + thrown.expect(NumberFormatException.class); + PubsubClient.extractTimestamp(null, "not-a-date", null); + } + + @Test + public void timestampLabelWithNullAttributesThrowsError() { + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel"); + PubsubClient.extractTimestamp("myLabel", null, null); + } + + @Test + public void timestampLabelSetWithMissingAttributeThrowsError() { + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel"); + Map map = ImmutableMap.of("otherLabel", "whatever"); + PubsubClient.extractTimestamp("myLabel", null, map); + } + + @Test + public void timestampLabelParsesMillisecondsSinceEpoch() { + long time = 1446162101123L; + Map map = ImmutableMap.of("myLabel", String.valueOf(time)); + long timestamp = PubsubClient.extractTimestamp("myLabel", null, map); + assertEquals(time, timestamp); + } + + @Test + public void timestampLabelParsesRfc3339Seconds() { + roundTripRfc339("2015-10-29T23:41:41Z"); + } + + @Test + public void timestampLabelParsesRfc3339Tenths() { + roundTripRfc339("2015-10-29T23:41:41.1Z"); + } + + @Test + public void timestampLabelParsesRfc3339Hundredths() { + roundTripRfc339("2015-10-29T23:41:41.12Z"); + } + + @Test + public void timestampLabelParsesRfc3339Millis() { + roundTripRfc339("2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339Micros() { + // Note: micros part 456/1000 is dropped. + truncatedRfc339("2015-10-29T23:41:41.123456Z", "2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339MicrosRounding() { + // Note: micros part 999/1000 is dropped, not rounded up. + truncatedRfc339("2015-10-29T23:41:41.123999Z", "2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelWithInvalidFormatThrowsError() { + thrown.expect(NumberFormatException.class); + parse("not-a-timestamp"); + } + + @Test + public void timestampLabelWithInvalidFormat2ThrowsError() { + thrown.expect(NumberFormatException.class); + parse("null"); + } + + @Test + public void timestampLabelWithInvalidFormat3ThrowsError() { + thrown.expect(NumberFormatException.class); + parse("2015-10"); + } + + @Test + public void timestampLabelParsesRfc3339WithSmallYear() { + // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted + // This is therefore a "small year" until this difference is reconciled. + roundTripRfc339("1582-10-15T01:23:45.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339WithLargeYear() { + // Year 9999 in range. + roundTripRfc339("9999-10-29T23:41:41.123999Z"); + } + + @Test + public void timestampLabelRfc3339WithTooLargeYearThrowsError() { + thrown.expect(NumberFormatException.class); + // Year 10000 out of range. + parse("10000-10-29T23:41:41.123999Z"); + } + + // + // Paths + // + + @Test + public void projectPathFromIdWellFormed() { + ProjectPath path = PubsubClient.projectPathFromId("test"); + assertEquals("projects/test", path.getPath()); + } + + @Test + public void subscriptionPathFromNameWellFormed() { + SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something"); + assertEquals("projects/test/subscriptions/something", path.getPath()); + assertEquals("/subscriptions/test/something", path.getV1Beta1Path()); + } + + @Test + public void topicPathFromNameWellFormed() { + TopicPath path = PubsubClient.topicPathFromName("test", "something"); + assertEquals("projects/test/topics/something", path.getPath()); + assertEquals("/topics/test/something", path.getV1Beta1Path()); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java new file mode 100644 index 000000000000..189049c07ea4 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.Hashing; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import com.google.pubsub.v1.PublishRequest; +import com.google.pubsub.v1.PublishResponse; +import com.google.pubsub.v1.PublisherGrpc; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.PullRequest; +import com.google.pubsub.v1.PullResponse; +import com.google.pubsub.v1.ReceivedMessage; +import com.google.pubsub.v1.SubscriberGrpc; + +import io.grpc.ManagedChannel; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubGrpcClient. + */ +public class PubsubGrpcClientTest { + private ManagedChannel mockChannel; + private GoogleCredentials mockCredentials; + private PublisherGrpc.PublisherBlockingStub mockPublisherStub; + private SubscriberGrpc.SubscriberBlockingStub mockSubscriberStub; + + private PubsubClient client; + + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long PUB_TIME = 3456L; + private static final long MESSAGE_TIME = 6789L; + private static final String TIMESTAMP_LABEL = "timestamp"; + private static final String ID_LABEL = "id"; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String CUSTOM_ID = + Hashing.murmur3_128().hashBytes(DATA.getBytes()).toString(); + private static final String ACK_ID = "testAckId"; + + @Before + public void setup() throws IOException { + mockChannel = Mockito.mock(ManagedChannel.class); + mockCredentials = Mockito.mock(GoogleCredentials.class); + mockPublisherStub = + Mockito.mock(PublisherGrpc.PublisherBlockingStub.class, Mockito.RETURNS_DEEP_STUBS); + mockSubscriberStub = + Mockito.mock(SubscriberGrpc.SubscriberBlockingStub.class, Mockito.RETURNS_DEEP_STUBS); + client = new PubsubGrpcClient(TIMESTAMP_LABEL, ID_LABEL, 0, mockChannel, + mockCredentials, mockPublisherStub, mockSubscriberStub); + } + + @After + public void teardown() throws IOException { + client.close(); + client = null; + mockChannel = null; + mockCredentials = null; + mockPublisherStub = null; + mockSubscriberStub = null; + } + + @Test + public void pullOneMessage() throws IOException { + String expectedSubscription = SUBSCRIPTION.getPath(); + PullRequest expectedRequest = + PullRequest.newBuilder() + .setSubscription(expectedSubscription) + .setReturnImmediately(true) + .setMaxMessages(10) + .build(); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(PUB_TIME / 1000) + .setNanos((int) (PUB_TIME % 1000) * 1000) + .build(); + PubsubMessage expectedPubsubMessage = + PubsubMessage.newBuilder() + .setMessageId(MESSAGE_ID) + .setData( + ByteString.copyFrom(DATA.getBytes())) + .setPublishTime(timestamp) + .putAllAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, + String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)) + .build(); + ReceivedMessage expectedReceivedMessage = + ReceivedMessage.newBuilder() + .setMessage(expectedPubsubMessage) + .setAckId(ACK_ID) + .build(); + PullResponse expectedResponse = + PullResponse.newBuilder() + .addAllReceivedMessages(ImmutableList.of(expectedReceivedMessage)) + .build(); + Mockito.when(mockSubscriberStub.pull(expectedRequest)) + .thenReturn(expectedResponse); + List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); + assertEquals(1, acutalMessages.size()); + IncomingMessage actualMessage = acutalMessages.get(0); + assertEquals(ACK_ID, actualMessage.ackId); + assertEquals(DATA, new String(actualMessage.elementBytes)); + assertEquals(CUSTOM_ID, new String(actualMessage.recordId)); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + } + + @Test + public void publishOneMessage() throws IOException { + String expectedTopic = TOPIC.getPath(); + PubsubMessage expectedPubsubMessage = + PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(DATA.getBytes())) + .putAllAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)) + .build(); + PublishRequest expectedRequest = + PublishRequest.newBuilder() + .setTopic(expectedTopic) + .addAllMessages( + ImmutableList.of(expectedPubsubMessage)) + .build(); + PublishResponse expectedResponse = + PublishResponse.newBuilder() + .addAllMessageIds(ImmutableList.of(MESSAGE_ID)) + .build(); + Mockito.when(mockPublisherStub.publish(expectedRequest)) + .thenReturn(expectedResponse); + OutgoingMessage actualMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); + assertEquals(1, n); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java new file mode 100644 index 000000000000..7d8513b725b1 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubTestClient. + */ +public class PubsubTestClientTest { + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long MESSAGE_TIME = 6789L; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String ACK_ID = "testAckId"; + private static final int ACK_TIMEOUT_S = 60; + + @Test + public void pullOneMessage() throws IOException { + IncomingMessage expectedIncomingMessage = + new IncomingMessage(DATA.getBytes(), MESSAGE_TIME, REQ_TIME, ACK_ID, MESSAGE_ID.getBytes()); + try (PubsubTestClient client = + new PubsubTestClient(null, SUBSCRIPTION, ACK_TIMEOUT_S, null, + Lists.newArrayList(expectedIncomingMessage))) { + long now = REQ_TIME; + client.advanceTo(now); + List incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage, incomingMessages.get(0)); + // Timeout on ACK. + now += (ACK_TIMEOUT_S + 10) * 1000; + client.advanceTo(now); + incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage.withRequestTime(now), incomingMessages.get(0)); + now += 10 * 1000; + client.advanceTo(now); + // Extend ack + client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20); + // Timeout on extended ACK + now += 30 * 1000; + client.advanceTo(now); + incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage.withRequestTime(now), incomingMessages.get(0)); + // Extend ack + client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20); + // Ack + now += 15 * 1000; + client.advanceTo(now); + client.acknowledge(SUBSCRIPTION, ImmutableList.of(ACK_ID)); + } + } + + @Test + public void publishOneMessage() throws IOException { + OutgoingMessage expectedOutgoingMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + try (PubsubTestClient client = + new PubsubTestClient(TOPIC, null, ACK_TIMEOUT_S, + Sets.newHashSet(expectedOutgoingMessage), null)) { + client.publish(TOPIC, ImmutableList.of(expectedOutgoingMessage)); + } + } +} From 7ecef4e688e6877c256da7ceb6bd5347f2409536 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 May 2016 13:22:59 -0700 Subject: [PATCH 12/29] Create runners/core module for artifact org.apache.beam:runners-core This is strictly creating the module and moving one easy class to it. Many of the utilities in org.apache.beam.util and subpackages should move as developments allow. --- pom.xml | 6 + runners/core-java/pom.xml | 208 ++++++++++++++++++ .../GroupAlsoByWindowViaWindowSetDoFn.java | 9 +- runners/core-java/src/test/java/.placeholder | 0 runners/direct-java/pom.xml | 21 ++ .../direct/GroupByKeyEvaluatorFactory.java | 2 +- runners/flink/runner/pom.xml | 11 + .../FlinkGroupAlsoByWindowWrapper.java | 8 +- runners/pom.xml | 1 + 9 files changed, 260 insertions(+), 6 deletions(-) create mode 100644 runners/core-java/pom.xml rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/GroupAlsoByWindowViaWindowSetDoFn.java (91%) create mode 100644 runners/core-java/src/test/java/.placeholder diff --git a/pom.xml b/pom.xml index b0b258bb8318..27787dab7ee4 100644 --- a/pom.xml +++ b/pom.xml @@ -194,6 +194,12 @@ ${project.version} + + org.apache.beam + runners-core + ${project.version} + + org.apache.beam google-cloud-dataflow-java-runner diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml new file mode 100644 index 000000000000..b6f6f2986925 --- /dev/null +++ b/runners/core-java/pom.xml @@ -0,0 +1,208 @@ + + + + + 4.0.0 + + + org.apache.beam + runners-parent + 0.1.0-incubating-SNAPSHOT + ../pom.xml + + + runners-core + Apache Beam :: Runners :: Core + Beam Runners Core provides utilities to aid runner authors. + + jar + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-dependency-plugin + + + analyze-only + + true + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + + + + + org.apache.maven.plugins + maven-source-plugin + + + + org.apache.maven.plugins + maven-javadoc-plugin + + Beam Runners Core utilities ${project.version} API + Beam Runners Core utilities for Java, version ${project.version} + ../javadoc/overview.html + + org.apache.beam.runners.core + false + true + ]]> + + + + + jar + + package + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + com.google.common + org.apache.beam.sdk.repackaged.com.google.common + + + com.google.thirdparty + org.apache.beam.sdk.repackaged.com.google.thirdparty + + + + + + + + bundle-rest-without-repackaging + package + + shade + + + true + ${project.artifactId}-bundled-${project.version} + + + com.google.guava:guava + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + org.jacoco + jacoco-maven-plugin + + + + + + + + org.apache.beam + java-sdk-all + + + + + + + org.hamcrest + hamcrest-all + test + + + + junit + junit + test + + + + org.slf4j + slf4j-jdk14 + test + + + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java similarity index 91% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index c24f6daac371..73244f7eca21 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -15,14 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; +import org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.ReduceFnRunner; +import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.SystemReduceFn; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/test/java/.placeholder b/runners/core-java/src/test/java/.placeholder new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 292cc56c8c5e..5c7dc38b2407 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -243,6 +243,27 @@ org.apache.beam java-sdk-all + ${project.version} + + + + org.apache.beam + runners-core + ${project.version} + + + + com.google.http-client + google-http-client + ${google-clients.version} + + + + com.google.guava + guava-jdk5 + + diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java index 874ec1780bc5..9a08996be215 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle; import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle; import org.apache.beam.runners.direct.StepTransformResult.Builder; @@ -32,7 +33,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItemCoder; diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index ab8d266b28fb..a1d53703ad70 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -99,6 +99,17 @@ + + org.apache.beam + runners-core + + + org.slf4j + slf4j-jdk14 + + + + org.apache.beam google-cloud-dataflow-java-runner diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 8d9744fb15c2..0306aa1ac9d1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.SystemReduceFn; @@ -83,7 +83,7 @@ * This class is the key class implementing all the windowing/triggering logic of Apache Beam. * To provide full compatibility and support for all the windowing/triggering combinations offered by * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in - * ({@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}. + * ({@link org.apache.beam.runners.core.GroupAlsoByWindowsDoFn}. *

* In a nutshell, when the execution arrives to this operator, we expect to have a stream already * grouped by key. Each of the elements that enter here, registers a timer @@ -95,7 +95,7 @@ * When a watermark arrives, all the registered timers are checked to see which ones are ready to * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers} - * list, and are fed into the {@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn} + * list, and are fed into the {@link org.apache.beam.runners.core.GroupAlsoByWindowsDoFn} * for furhter processing. */ public class FlinkGroupAlsoByWindowWrapper @@ -253,7 +253,7 @@ public void open() throws Exception { } /** - * Create the adequate {@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}, + * Create the adequate {@link org.apache.beam.runners.core.GroupAlsoByWindowsDoFn}, * if not already created. * If a {@link org.apache.beam.sdk.transforms.Combine.KeyedCombineFn} was provided, then * a function with that combiner is created, so that elements are combined as they arrive. This is diff --git a/runners/pom.xml b/runners/pom.xml index 74812e8d0cc7..d2d68dfca132 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -35,6 +35,7 @@ Apache Beam :: Runners + core-java direct-java flink spark From 658e6099dae849679c70d76dc85ba0a0ef6c2e96 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 May 2016 20:09:16 -0700 Subject: [PATCH 13/29] Fix direct runner pom & deps --- runners/direct-java/pom.xml | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 5c7dc38b2407..79cf517d5805 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -243,27 +243,11 @@ org.apache.beam java-sdk-all - ${project.version} org.apache.beam runners-core - ${project.version} - - - - com.google.http-client - google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - From dc0f6bb992cc5c119ae5098241e0ad237747cee5 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 2 May 2016 10:03:43 -0700 Subject: [PATCH 14/29] Move ReadyCheckingSideInputReader to util This SideInputReader allows callers to check for a side input being available before attempting to read the contents --- .../direct/InProcessEvaluationContext.java | 11 +----- .../direct/InProcessSideInputContainer.java | 2 +- .../InProcessSideInputContainerTest.java | 2 +- .../util/ReadyCheckingSideInputReader.java | 34 +++++++++++++++++++ 4 files changed, 37 insertions(+), 12 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java index d4f891ee450c..bcf85da9d52d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.util.ExecutionContext; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowedValue; @@ -333,16 +334,6 @@ public ReadyCheckingSideInputReader createSideInputReader( return sideInputContainer.createReaderForViews(sideInputs); } - /** - * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has - * had its contents set in a window. - */ - static interface ReadyCheckingSideInputReader extends SideInputReader { - /** - * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}. - */ - boolean isReady(PCollectionView view, BoundedWindow window); - } /** * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java index b01cd484c2e2..78889dc69689 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java @@ -19,10 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.PCollectionViewWindow; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java index d8a78f23d3e8..8f89e707b9bb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.doAnswer; -import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.PCollectionViews; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java new file mode 100644 index 000000000000..cb38a55a0da4 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.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.sdk.util; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has + * had its contents set in a window. + */ +public interface ReadyCheckingSideInputReader extends SideInputReader { + /** + * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}. + */ + boolean isReady(PCollectionView view, BoundedWindow window); +} + From 2366fa5616bb09ba3e7b1dc312baa92440154f86 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 2 May 2016 10:04:20 -0700 Subject: [PATCH 15/29] Add PushbackSideInputDoFnRunner This DoFnRunner wraps a DoFnRunner and provides an additional method to process an element in all the windows where all side inputs are ready, returning any elements that it could not process. --- .../sdk/util/PushbackSideInputDoFnRunner.java | 115 +++++++++ .../sdk/util/IdentitySideInputWindowFn.java | 54 ++++ .../util/PushbackSideInputDoFnRunnerTest.java | 234 ++++++++++++++++++ 3 files changed, 403 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java new file mode 100644 index 000000000000..4eeedf65a149 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning + * them via the {@link #processElementInReadyWindows(WindowedValue)}. + */ +class PushbackSideInputDoFnRunner implements DoFnRunner { + private final DoFnRunner underlying; + private final Collection> views; + private final ReadyCheckingSideInputReader sideInputReader; + + private Set notReadyWindows; + + public static PushbackSideInputDoFnRunner create( + DoFnRunner underlying, + Collection> views, + ReadyCheckingSideInputReader sideInputReader) { + return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader); + } + + private PushbackSideInputDoFnRunner( + DoFnRunner underlying, + Collection> views, + ReadyCheckingSideInputReader sideInputReader) { + this.underlying = underlying; + this.views = views; + this.sideInputReader = sideInputReader; + } + + @Override + public void startBundle() { + notReadyWindows = new HashSet<>(); + underlying.startBundle(); + } + + /** + * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element + * for each window the element is in that is ready. + * + * @param elem the element to process in all ready windows + * @return each element that could not be processed because it requires a side input window + * that is not ready. + */ + public Iterable> processElementInReadyWindows(WindowedValue elem) { + if (views.isEmpty()) { + processElement(elem); + return Collections.emptyList(); + } + ImmutableList.Builder> pushedBack = ImmutableList.builder(); + for (WindowedValue windowElem : elem.explodeWindows()) { + BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows()); + boolean isReady = !notReadyWindows.contains(mainInputWindow); + for (PCollectionView view : views) { + BoundedWindow sideInputWindow = + view.getWindowingStrategyInternal() + .getWindowFn() + .getSideInputWindow(mainInputWindow); + if (!sideInputReader.isReady(view, sideInputWindow)) { + isReady = false; + break; + } + } + if (isReady) { + processElement(windowElem); + } else { + notReadyWindows.add(mainInputWindow); + pushedBack.add(windowElem); + } + } + return pushedBack.build(); + } + + @Override + public void processElement(WindowedValue elem) { + underlying.processElement(elem); + } + + /** + * Call the underlying {@link DoFnRunner#finishBundle()}. + */ + @Override + public void finishBundle() { + notReadyWindows = null; + underlying.finishBundle(); + } +} + diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java new file mode 100644 index 000000000000..ecab6f8eb6c5 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; + +import java.util.Collection; + +/** + * A {@link WindowFn} for use during tests that returns the input window for calls to + * {@link #getSideInputWindow(BoundedWindow)}. + */ +class IdentitySideInputWindowFn extends NonMergingWindowFn { + @Override + public Collection assignWindows(WindowFn.AssignContext c) + throws Exception { + return (Collection) c.windows(); + } + + @Override + public boolean isCompatible(WindowFn other) { + return true; + } + + @Override + public Coder windowCoder() { + // not used + return (Coder) GlobalWindow.Coder.INSTANCE; + } + + @Override + public BoundedWindow getSideInputWindow(BoundedWindow window) { + return window; + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java new file mode 100644 index 000000000000..88851187bb65 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.when; + +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.ImmutableList; + +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; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for {@link PushbackSideInputDoFnRunner}. + */ +@RunWith(JUnit4.class) +public class PushbackSideInputDoFnRunnerTest { + @Mock private ReadyCheckingSideInputReader reader; + private TestDoFnRunner underlying; + private PCollectionView singletonView; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + TestPipeline p = TestPipeline.create(); + PCollection created = p.apply(Create.of(1, 2, 3)); + singletonView = + created + .apply(Window.into(new IdentitySideInputWindowFn())) + .apply(Sum.integersGlobally().asSingletonView()); + + underlying = new TestDoFnRunner<>(); + } + + private PushbackSideInputDoFnRunner createRunner( + ImmutableList> views) { + PushbackSideInputDoFnRunner runner = + PushbackSideInputDoFnRunner.create(underlying, views, reader); + runner.startBundle(); + return runner; + } + + @Test + public void startFinishBundleDelegates() { + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + assertThat(underlying.started, is(true)); + assertThat(underlying.finished, is(false)); + runner.finishBundle(); + assertThat(underlying.finished, is(true)); + } + + @Test + public void processElementSideInputNotReady() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(false); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + WindowedValue oneWindow = + WindowedValue.of( + 2, + new Instant(-2), + new IntervalWindow(new Instant(-500L), new Instant(0L)), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> oneWindowPushback = + runner.processElementInReadyWindows(oneWindow); + assertThat(oneWindowPushback, containsInAnyOrder(oneWindow)); + assertThat(underlying.inputElems, Matchers.>emptyIterable()); + } + + @Test + public void processElementSideInputNotReadyMultipleWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(false); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows())); + assertThat(underlying.inputElems, Matchers.>emptyIterable()); + } + + @Test + public void processElementSideInputNotReadySomeWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE))) + .thenReturn(false); + when( + reader.isReady( + Mockito.eq(singletonView), + org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE)))) + .thenReturn(true); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L)); + IntervalWindow bigWindow = + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)); + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE), + PaneInfo.NO_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat( + multiWindowPushback, + containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L)))); + assertThat(underlying.inputElems, + containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING), + WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING))); + } + + @Test + public void processElementSideInputReadyAllWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(true); + + ImmutableList> views = ImmutableList.>of(singletonView); + PushbackSideInputDoFnRunner runner = createRunner(views); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, emptyIterable()); + assertThat(underlying.inputElems, + containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray())); + } + + @Test + public void processElementNoSideInputs() { + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of()); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, emptyIterable()); + assertThat(underlying.inputElems, containsInAnyOrder(multiWindow)); + } + + private static class TestDoFnRunner implements DoFnRunner { + List> inputElems; + private boolean started = false; + private boolean finished = false; + + @Override + public void startBundle() { + started = true; + inputElems = new ArrayList<>(); + } + + @Override + public void processElement(WindowedValue elem) { + inputElems.add(elem); + } + + @Override + public void finishBundle() { + finished = true; + } + } +} From 8529c69a1ce9ae91363d3a8e93659eba019c0379 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 3 May 2016 17:55:11 -0700 Subject: [PATCH 16/29] [BEAM-48] Refactor BigQueryServices to support extract and query jobs --- .../org/apache/beam/sdk/io/BigQueryIO.java | 40 ++++- .../beam/sdk/util/BigQueryServices.java | 40 +++-- .../beam/sdk/util/BigQueryServicesImpl.java | 148 +++++++++++------- .../apache/beam/sdk/io/BigQueryIOTest.java | 79 +++++++--- .../sdk/util/BigQueryServicesImplTest.java | 79 ++++++---- 5 files changed, 252 insertions(+), 134 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java index 4a0bdacb6970..77852989dd97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java @@ -47,7 +47,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.BigQueryServices; -import org.apache.beam.sdk.util.BigQueryServices.LoadService; +import org.apache.beam.sdk.util.BigQueryServices.JobService; import org.apache.beam.sdk.util.BigQueryServicesImpl; import org.apache.beam.sdk.util.BigQueryTableInserter; import org.apache.beam.sdk.util.BigQueryTableRowIterator; @@ -68,7 +68,9 @@ import com.google.api.client.json.JsonFactory; import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.QueryRequest; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; @@ -1168,6 +1170,10 @@ private static class BigQueryWriteOperation extends FileBasedWriteOperation writerResults, PipelineOptions options } if (!tempFiles.isEmpty()) { load( - bigQuerySink.bqServices.getLoadService(bqOptions), + bigQuerySink.bqServices.getJobService(bqOptions), bigQuerySink.jobIdToken, fromJsonString(bigQuerySink.jsonTable, TableReference.class), tempFiles, @@ -1215,7 +1221,7 @@ public void finalize(Iterable writerResults, PipelineOptions options *

If a load job failed, it will try another load job with a different job id. */ private void load( - LoadService loadService, + JobService jobService, String jobIdPrefix, TableReference ref, List gcsUris, @@ -1238,8 +1244,9 @@ private void load( LOG.info("Previous load jobs failed, retrying."); } LOG.info("Starting BigQuery load job: {}", jobId); - loadService.startLoadJob(jobId, loadConfig); - BigQueryServices.Status jobStatus = loadService.pollJobStatus(projectId, jobId); + jobService.startLoadJob(jobId, loadConfig); + Status jobStatus = parseStatus( + jobService.pollJob(projectId, jobId, MAX_JOB_STATUS_POLL_RETRIES)); switch (jobStatus) { case SUCCEEDED: return; @@ -1669,6 +1676,29 @@ public PDone apply(PCollection input) { } } + /** + * Status of a BigQuery job or request. + */ + enum Status { + SUCCEEDED, + FAILED, + UNKNOWN, + } + + private static Status parseStatus(@Nullable Job job) { + if (job == null) { + return Status.UNKNOWN; + } + JobStatus status = job.getStatus(); + if (status.getErrorResult() != null) { + return Status.FAILED; + } else if (status.getErrors() != null && !status.getErrors().isEmpty()) { + return Status.FAILED; + } else { + return Status.SUCCEEDED; + } + } + private static String toJsonString(Object item) { if (item == null) { return null; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java index 0fecfdcfd8a8..b12e049a4dab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java @@ -19,7 +19,10 @@ import org.apache.beam.sdk.options.BigQueryOptions; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; import java.io.IOException; import java.io.Serializable; @@ -30,33 +33,38 @@ public interface BigQueryServices extends Serializable { /** - * Status of a BigQuery job or request. + * Returns a real, mock, or fake {@link JobService}. */ - enum Status { - SUCCEEDED, - FAILED, - UNKNOWN, - } - - /** - * Returns a real, mock, or fake {@link LoadService}. - */ - public LoadService getLoadService(BigQueryOptions bqOptions); + public JobService getJobService(BigQueryOptions bqOptions); /** * An interface for the Cloud BigQuery load service. */ - public interface LoadService { + public interface JobService { /** - * Start a BigQuery load job. + * Starts a BigQuery load job. */ - public void startLoadJob(String jobId, JobConfigurationLoad loadConfig) + void startLoadJob(String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException; /** - * Poll the status of a BigQuery load job. + * Start a BigQuery extract job. + */ + void startExtractJob(String jobId, JobConfigurationExtract extractConfig) + throws InterruptedException, IOException; + + /** + * Start a BigQuery extract job. + */ + void startQueryJob(String jobId, JobConfigurationQuery query, boolean dryRun) + throws IOException, InterruptedException; + + /** + * Waits for the job is Done, and returns the job. + * + *

Returns null if the {@code maxAttempts} retries reached. */ - public Status pollJobStatus(String projectId, String jobId) + Job pollJob(String projectId, String jobId, int maxAttempts) throws InterruptedException, IOException; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java index 0502c6e9f634..2bfe84f75452 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java @@ -26,10 +26,11 @@ import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfiguration; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatus; -import com.google.api.services.bigquery.model.TableReference; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; @@ -45,37 +46,34 @@ */ public class BigQueryServicesImpl implements BigQueryServices { - // The maximum number of attempts to execute a load job RPC. - private static final int MAX_LOAD_JOB_RPC_ATTEMPTS = 10; + // The maximum number of attempts to execute a BigQuery RPC. + private static final int MAX_RPC_ATTEMPTS = 10; - // The initial backoff for executing a load job RPC. - private static final long INITIAL_LOAD_JOB_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); - // The maximum number of retries to poll the status of a load job. - // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery load job finishes. - private static final int MAX_LOAD_JOB_POLL_RETRIES = Integer.MAX_VALUE; + // The initial backoff for executing a BigQuery RPC. + private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); - // The initial backoff for polling the status of a load job. - private static final long INITIAL_LOAD_JOB_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(60); + // The initial backoff for polling the status of a BigQuery job. + private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(60); @Override - public LoadService getLoadService(BigQueryOptions options) { - return new LoadServiceImpl(options); + public JobService getJobService(BigQueryOptions options) { + return new JobServiceImpl(options); } @VisibleForTesting - static class LoadServiceImpl implements BigQueryServices.LoadService { - private static final Logger LOG = LoggerFactory.getLogger(LoadServiceImpl.class); + static class JobServiceImpl implements BigQueryServices.JobService { + private static final Logger LOG = LoggerFactory.getLogger(JobServiceImpl.class); private final ApiErrorExtractor errorExtractor; private final Bigquery client; @VisibleForTesting - LoadServiceImpl(Bigquery client) { + JobServiceImpl(Bigquery client) { this.errorExtractor = new ApiErrorExtractor(); this.client = client; } - private LoadServiceImpl(BigQueryOptions options) { + private JobServiceImpl(BigQueryOptions options) { this.errorExtractor = new ApiErrorExtractor(); this.client = Transport.newBigQueryClient(options).build(); } @@ -83,7 +81,7 @@ private LoadServiceImpl(BigQueryOptions options) { /** * {@inheritDoc} * - *

Retries the RPC for at most {@code MAX_LOAD_JOB_RPC_ATTEMPTS} times until it succeeds. + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. * * @throws IOException if it exceeds max RPC retries. */ @@ -91,47 +89,84 @@ private LoadServiceImpl(BigQueryOptions options) { public void startLoadJob( String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_LOAD_JOB_RPC_ATTEMPTS, INITIAL_LOAD_JOB_RPC_BACKOFF_MILLIS); - startLoadJob(jobId, loadConfig, Sleeper.DEFAULT, backoff); + Job job = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setProjectId(loadConfig.getDestinationTable().getProjectId()); + jobRef.setJobId(jobId); + job.setJobReference(jobRef); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setLoad(loadConfig); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); } /** * {@inheritDoc} * - *

Retries the poll request for at most {@code MAX_LOAD_JOB_POLL_RETRIES} times - * until the job is DONE. + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + * + * @throws IOException if it exceeds max RPC retries. */ @Override - public Status pollJobStatus(String projectId, String jobId) throws InterruptedException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_LOAD_JOB_POLL_RETRIES, INITIAL_LOAD_JOB_POLL_BACKOFF_MILLIS); - return pollJobStatus(projectId, jobId, Sleeper.DEFAULT, backoff); - } - - @VisibleForTesting - void startLoadJob( - String jobId, - JobConfigurationLoad loadConfig, - Sleeper sleeper, - BackOff backoff) + public void startExtractJob(String jobId, JobConfigurationExtract extractConfig) throws InterruptedException, IOException { - TableReference ref = loadConfig.getDestinationTable(); - String projectId = ref.getProjectId(); + Job job = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setProjectId(extractConfig.getSourceTable().getProjectId()); + jobRef.setJobId(jobId); + job.setJobReference(jobRef); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setExtract(extractConfig); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); + } + /** + * {@inheritDoc} + * + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + * + * @throws IOException if it exceeds max RPC retries. + */ + @Override + public void startQueryJob(String jobId, JobConfigurationQuery queryConfig, boolean dryRun) + throws IOException, InterruptedException { Job job = new Job(); JobReference jobRef = new JobReference(); - jobRef.setProjectId(projectId); + jobRef.setProjectId(queryConfig.getDestinationTable().getProjectId()); jobRef.setJobId(jobId); job.setJobReference(jobRef); - JobConfiguration config = new JobConfiguration(); - config.setLoad(loadConfig); - job.setConfiguration(config); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setQuery(queryConfig); + jobConfig.setDryRun(dryRun); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); + } + private static void startJob(Job job, + ApiErrorExtractor errorExtractor, + Bigquery client) throws IOException, InterruptedException { + BackOff backoff = + new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff); + } + + @VisibleForTesting + static void startJob( + Job job, + ApiErrorExtractor errorExtractor, + Bigquery client, + Sleeper sleeper, + BackOff backoff) + throws InterruptedException, IOException { + JobReference jobRef = job.getJobReference(); Exception lastException = null; do { try { - client.jobs().insert(projectId, job).execute(); + client.jobs().insert(jobRef.getProjectId(), job).execute(); return; // SUCCEEDED } catch (GoogleJsonResponseException e) { if (errorExtractor.itemAlreadyExists(e)) { @@ -149,27 +184,30 @@ void startLoadJob( throw new IOException( String.format( "Unable to insert job: %s, aborting after %d retries.", - jobId, MAX_LOAD_JOB_RPC_ATTEMPTS), + jobRef.getJobId(), MAX_RPC_ATTEMPTS), lastException); } + @Override + public Job pollJob(String projectId, String jobId, int maxAttempts) + throws InterruptedException { + BackOff backoff = new AttemptBoundedExponentialBackOff( + maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS); + return pollJob(projectId, jobId, Sleeper.DEFAULT, backoff); + } + @VisibleForTesting - Status pollJobStatus( + Job pollJob( String projectId, String jobId, Sleeper sleeper, BackOff backoff) throws InterruptedException { do { try { - JobStatus status = client.jobs().get(projectId, jobId).execute().getStatus(); + Job job = client.jobs().get(projectId, jobId).execute(); + JobStatus status = job.getStatus(); if (status != null && status.getState() != null && status.getState().equals("DONE")) { - if (status.getErrorResult() != null) { - return Status.FAILED; - } else if (status.getErrors() != null && !status.getErrors().isEmpty()) { - return Status.FAILED; - } else { - return Status.SUCCEEDED; - } + return job; } // The job is not DONE, wait longer and retry. } catch (IOException e) { @@ -177,16 +215,16 @@ Status pollJobStatus( LOG.warn("Ignore the error and retry polling job status.", e); } } while (nextBackOff(sleeper, backoff)); - LOG.warn("Unable to poll job status: {}, aborting after {} retries.", - jobId, MAX_LOAD_JOB_POLL_RETRIES); - return Status.UNKNOWN; + LOG.warn("Unable to poll job status: {}, aborting after reached max retries.", jobId); + return null; } /** * Identical to {@link BackOffUtils#next} but without checked IOException. * @throws InterruptedException */ - private boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException { + private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) + throws InterruptedException { try { return BackOffUtils.next(sleeper, backoff); } catch (IOException e) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java index 63ff22c63cc5..7998fc70b7df 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.TableRowJsonCoder; +import org.apache.beam.sdk.io.BigQueryIO.Status; import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.options.BigQueryOptions; @@ -39,16 +40,21 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.BigQueryServices; -import org.apache.beam.sdk.util.BigQueryServices.Status; import org.apache.beam.sdk.util.CoderUtils; import com.google.api.client.util.Data; +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.hamcrest.Matchers; import org.junit.Assert; @@ -66,6 +72,7 @@ import java.io.File; import java.io.FileFilter; import java.io.IOException; +import java.util.Map; /** * Tests for BigQueryIO. @@ -73,23 +80,28 @@ @RunWith(JUnit4.class) public class BigQueryIOTest { + // Status.UNKNOWN maps to null + private static final Map JOB_STATUS_MAP = ImmutableMap.of( + Status.SUCCEEDED, new Job().setStatus(new JobStatus()), + Status.FAILED, new Job().setStatus(new JobStatus().setErrorResult(new ErrorProto()))); + private static class FakeBigQueryServices implements BigQueryServices { - private Object[] startLoadJobReturns; + private Object[] startJobReturns; private Object[] pollJobStatusReturns; /** - * Sets the return values for the mock {@link LoadService#startLoadJob}. + * Sets the return values for the mock {@link JobService#startLoadJob}. * *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. */ private FakeBigQueryServices startLoadJobReturns(Object... startLoadJobReturns) { - this.startLoadJobReturns = startLoadJobReturns; + this.startJobReturns = startLoadJobReturns; return this; } /** - * Sets the return values for the mock {@link LoadService#pollJobStatus}. + * Sets the return values for the mock {@link JobService#pollJobStatus}. * *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. */ @@ -99,19 +111,19 @@ private FakeBigQueryServices pollJobStatusReturns(Object... pollJobStatusReturns } @Override - public LoadService getLoadService(BigQueryOptions bqOptions) { - return new FakeLoadService(startLoadJobReturns, pollJobStatusReturns); + public JobService getJobService(BigQueryOptions bqOptions) { + return new FakeLoadService(startJobReturns, pollJobStatusReturns); } - private static class FakeLoadService implements BigQueryServices.LoadService { + private static class FakeLoadService implements BigQueryServices.JobService { - private Object[] startLoadJobReturns; + private Object[] startJobReturns; private Object[] pollJobStatusReturns; private int startLoadJobCallsCount; private int pollJobStatusCallsCount; public FakeLoadService(Object[] startLoadJobReturns, Object[] pollJobStatusReturns) { - this.startLoadJobReturns = startLoadJobReturns; + this.startJobReturns = startLoadJobReturns; this.pollJobStatusReturns = pollJobStatusReturns; this.startLoadJobCallsCount = 0; this.pollJobStatusCallsCount = 0; @@ -120,35 +132,52 @@ public FakeLoadService(Object[] startLoadJobReturns, Object[] pollJobStatusRetur @Override public void startLoadJob(String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException { - if (startLoadJobCallsCount < startLoadJobReturns.length) { - Object ret = startLoadJobReturns[startLoadJobCallsCount++]; - if (ret instanceof IOException) { - throw (IOException) ret; + startJob(); + } + + @Override + public void startExtractJob(String jobId, JobConfigurationExtract extractConfig) + throws InterruptedException, IOException { + startJob(); + } + + @Override + public void startQueryJob(String jobId, JobConfigurationQuery query, boolean dryRun) + throws IOException, InterruptedException { + startJob(); + } + + @Override + public Job pollJob(String projectId, String jobId, int maxAttemps) + throws InterruptedException { + if (pollJobStatusCallsCount < pollJobStatusReturns.length) { + Object ret = pollJobStatusReturns[pollJobStatusCallsCount++]; + if (ret instanceof Status) { + return JOB_STATUS_MAP.get(ret); } else if (ret instanceof InterruptedException) { throw (InterruptedException) ret; } else { - return; + throw new RuntimeException("Unexpected return type: " + ret.getClass()); } } else { throw new RuntimeException( - "Exceeded expected number of calls: " + startLoadJobReturns.length); + "Exceeded expected number of calls: " + pollJobStatusReturns.length); } } - @Override - public Status pollJobStatus(String projectId, String jobId) throws InterruptedException { - if (pollJobStatusCallsCount < pollJobStatusReturns.length) { - Object ret = pollJobStatusReturns[pollJobStatusCallsCount++]; - if (ret instanceof Status) { - return (Status) ret; + private void startJob() throws IOException, InterruptedException { + if (startLoadJobCallsCount < startJobReturns.length) { + Object ret = startJobReturns[startLoadJobCallsCount++]; + if (ret instanceof IOException) { + throw (IOException) ret; } else if (ret instanceof InterruptedException) { throw (InterruptedException) ret; } else { - throw new RuntimeException("Unexpected return type: " + ret.getClass()); + return; } } else { throw new RuntimeException( - "Exceeded expected number of calls: " + pollJobStatusReturns.length); + "Exceeded expected number of calls: " + startJobReturns.length); } } } @@ -160,7 +189,7 @@ public Status pollJobStatus(String projectId, String jobId) throws InterruptedEx @Rule public TemporaryFolder testFolder = new TemporaryFolder(); @Mock - public BigQueryServices.LoadService mockBqLoadService; + public BigQueryServices.JobService mockBqLoadService; private BigQueryOptions bqOptions; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java index 74a2da2198ec..238deede2b86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.BigQueryServicesImpl.JobServiceImpl; import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; @@ -40,8 +41,10 @@ import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.TableReference; +import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.collect.ImmutableList; import org.junit.Before; @@ -91,11 +94,16 @@ public LowLevelHttpResponse execute() throws IOException { } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds. */ @Test public void testStartLoadJobSucceeds() throws IOException, InterruptedException { Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); @@ -108,20 +116,24 @@ public void testStartLoadJobSucceeds() throws IOException, InterruptedException Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds * with an already exist job. */ @Test public void testStartLoadJobSucceedsAlreadyExists() throws IOException, InterruptedException { + Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); + when(response.getStatusCode()).thenReturn(409); // 409 means already exists TableReference ref = new TableReference(); @@ -132,9 +144,7 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -142,11 +152,15 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds with a retry. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds with a retry. */ @Test public void testStartLoadJobRetry() throws IOException, InterruptedException { Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); // First response is 403 rate limited, second response has valid payload. when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); @@ -163,19 +177,18 @@ public void testStartLoadJobRetry() throws IOException, InterruptedException { Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); verify(response, times(2)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} succeeds. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} succeeds. */ @Test - public void testPollJobStatusSucceeds() throws IOException, InterruptedException { + public void testPollJobSucceeds() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus().setState("DONE")); @@ -183,22 +196,22 @@ public void testPollJobStatusSucceeds() throws IOException, InterruptedException when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); - assertEquals(BigQueryServices.Status.SUCCEEDED, status); + assertEquals(testJob, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} fails. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} fails. */ @Test - public void testPollJobStatusFailed() throws IOException, InterruptedException { + public void testPollJobFailed() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus().setState("DONE").setErrorResult(new ErrorProto())); @@ -206,22 +219,22 @@ public void testPollJobStatusFailed() throws IOException, InterruptedException { when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); - assertEquals(BigQueryServices.Status.FAILED, status); + assertEquals(testJob, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} returns UNKNOWN. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} returns UNKNOWN. */ @Test - public void testPollJobStatusUnknown() throws IOException, InterruptedException { + public void testPollJobUnknown() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus()); @@ -229,12 +242,12 @@ public void testPollJobStatusUnknown() throws IOException, InterruptedException when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.STOP_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.STOP_BACKOFF); - assertEquals(BigQueryServices.Status.UNKNOWN, status); + assertEquals(null, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); From 7a8b1cc1f8877ea7d4b9a8ec9b3b765804424a58 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 May 2016 21:11:39 -0700 Subject: [PATCH 17/29] Speed up non-release builds --- pom.xml | 141 +++++++++++++++++----------- runners/direct-java/pom.xml | 8 -- sdks/java/core/pom.xml | 180 ++++++++++++++++++------------------ sdks/pom.xml | 21 +++++ 4 files changed, 197 insertions(+), 153 deletions(-) diff --git a/pom.xml b/pom.xml index 27787dab7ee4..fda7a24bcd00 100644 --- a/pom.xml +++ b/pom.xml @@ -132,19 +132,79 @@ sdks - + runners/google-cloud-dataflow-java runners - + sdks/java/maven-archetypes examples + + + + release + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + javadoc + package + + jar + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + package + + jar-no-fork + + + + attach-test-sources + package + + test-jar-no-fork + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + default-test-jar + + test-jar + + + + + + + + + doclint-java8-disable @@ -154,6 +214,7 @@ -Xdoclint:-missing + src @@ -335,8 +396,8 @@ google-api-services-storage ${storage.version} - + com.google.guava guava-jdk5 @@ -349,8 +410,8 @@ google-api-services-datastore-protobuf ${datastore.version} - + com.google.guava guava-jdk5 @@ -413,8 +474,8 @@ google-api-services-dataflow ${dataflow.version} - + com.google.guava guava-jdk5 @@ -427,8 +488,8 @@ google-api-services-clouddebugger ${clouddebugger.version} - + com.google.guava guava-jdk5 @@ -496,6 +557,7 @@ test-jar test + org.apache.beam java-sdk-all @@ -605,7 +667,7 @@ -Xlint:-try -Xlint:-unchecked -Xlint:-varargs - + @@ -626,12 +688,6 @@ jar - - default-test-jar - - test-jar - - @@ -639,37 +695,12 @@ org.apache.maven.plugins maven-source-plugin 2.4 - - - attach-sources - package - - jar-no-fork - - - - attach-test-sources - package - - test-jar-no-fork - - - org.apache.maven.plugins maven-javadoc-plugin 2.10.3 - - - javadoc - package - - jar - - - @@ -723,11 +754,11 @@ - - - **/AutoValue_*.class - - + + + **/AutoValue_*.class + + @@ -761,8 +792,8 @@ 2.19.1 - + org.eclipse.m2e lifecycle-mapping @@ -796,7 +827,7 @@ - + @@ -810,7 +841,7 @@ - + diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 79cf517d5805..bec9b70a8ffb 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -150,14 +150,6 @@ - - - - jar - - package - - diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index a3b7cad0f61d..d1adf55883db 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - @@ -49,6 +49,75 @@ + + + + org.apache.maven.plugins + maven-javadoc-plugin + + Apache Beam SDK ${project.version} API + Apache Beam SDK for Java, version ${project.version} + ../javadoc/overview.html + + org.apache.beam.sdk + -exclude + org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess + ${dataflow.javadoc_opts} + false + true + ]]> + + + + https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/ + ${basedir}/../javadoc/apiclient-docs + + + http://avro.apache.org/docs/1.7.7/api/java/ + ${basedir}/../javadoc/avro-docs + + + https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/ + ${basedir}/../javadoc/bq-docs + + + https://cloud.google.com/datastore/docs/apis/javadoc/ + ${basedir}/../javadoc/datastore-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/ + ${basedir}/../javadoc/guava-docs + + + http://hamcrest.org/JavaHamcrest/javadoc/1.3/ + ${basedir}/../javadoc/hamcrest-docs + + + http://fasterxml.github.io/jackson-annotations/javadoc/2.7/ + ${basedir}/../javadoc/jackson-annotations-docs + + + http://fasterxml.github.io/jackson-databind/javadoc/2.7/ + ${basedir}/../javadoc/jackson-databind-docs + + + http://www.joda.org/joda-time/apidocs + ${basedir}/../javadoc/joda-docs + + + http://junit.sourceforge.net/javadoc/ + ${basedir}/../javadoc/junit-docs + + + https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/ + ${basedir}/../javadoc/oauth-docs + + + + + + + org.apache.maven.plugins @@ -60,7 +129,9 @@ maven-dependency-plugin - analyze-only + + analyze-only + true @@ -91,72 +162,6 @@ org.apache.maven.plugins maven-javadoc-plugin - - Apache Beam SDK ${project.version} API - Apache Beam SDK for Java, version ${project.version} - ../javadoc/overview.html - - org.apache.beam.sdk - -exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess ${dataflow.javadoc_opts} - false - true - ]]> - - - - https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/ - ${basedir}/../javadoc/apiclient-docs - - - http://avro.apache.org/docs/1.7.7/api/java/ - ${basedir}/../javadoc/avro-docs - - - https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/ - ${basedir}/../javadoc/bq-docs - - - https://cloud.google.com/datastore/docs/apis/javadoc/ - ${basedir}/../javadoc/datastore-docs - - - http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/ - ${basedir}/../javadoc/guava-docs - - - http://hamcrest.org/JavaHamcrest/javadoc/1.3/ - ${basedir}/../javadoc/hamcrest-docs - - - http://fasterxml.github.io/jackson-annotations/javadoc/2.7/ - ${basedir}/../javadoc/jackson-annotations-docs - - - http://fasterxml.github.io/jackson-databind/javadoc/2.7/ - ${basedir}/../javadoc/jackson-databind-docs - - - http://www.joda.org/joda-time/apidocs - ${basedir}/../javadoc/joda-docs - - - http://junit.sourceforge.net/javadoc/ - ${basedir}/../javadoc/junit-docs - - - https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/ - ${basedir}/../javadoc/oauth-docs - - - - - - - jar - - package - - @@ -188,9 +193,8 @@ - + com.google.common org.apache.beam.sdk.repackaged.com.google.common @@ -203,8 +207,8 @@ - + bundle-rest-without-repackaging package @@ -259,10 +263,9 @@ - + org.codehaus.mojo build-helper-maven-plugin @@ -295,8 +298,8 @@ google-auth-library-oauth2-http 0.3.1 - + com.google.guava guava-jdk5 @@ -315,8 +318,8 @@ grpc-pubsub-v1 ${pubsubgrpc.version} - + com.google.guava guava-jdk5 @@ -349,8 +352,8 @@ google-http-client - + com.google.http-client google-http-client-jackson @@ -456,8 +459,8 @@ joda-time - + org.tukaani xz diff --git a/sdks/pom.xml b/sdks/pom.xml index 808f0d33c8c5..0567f5756eee 100644 --- a/sdks/pom.xml +++ b/sdks/pom.xml @@ -38,4 +38,25 @@ java + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + default-test-jar + + test-jar + + + + + + + + \ No newline at end of file From 50012a4edce44bc57e6a9a22bb61f0f5dfe37a70 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 3 May 2016 13:22:13 -0700 Subject: [PATCH 18/29] Refactor CompletionCallbacks The default and timerful completion callbacks are identical, excepting their calls to evaluationContext.commitResult; factor that code into a common location. --- .../ExecutorServiceParallelExecutor.java | 51 +++++++++++-------- 1 file changed, 30 insertions(+), 21 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 18af36337692..9f26e5a51677 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -210,16 +210,20 @@ public void awaitCompletion() throws Throwable { } /** - * The default {@link CompletionCallback}. The default completion callback is used to complete - * transform evaluations that are triggered due to the arrival of elements from an upstream - * transform, or for a source transform. + * The base implementation of {@link CompletionCallback} that provides implementations for + * {@link #handleResult(CommittedBundle, InProcessTransformResult)} and + * {@link #handleThrowable(CommittedBundle, Throwable)}, given an implementation of + * {@link #getCommittedResult(CommittedBundle, InProcessTransformResult)}. */ - private class DefaultCompletionCallback implements CompletionCallback { + private abstract class CompletionCallbackBase implements CompletionCallback { + protected abstract CommittedResult getCommittedResult( + CommittedBundle inputBundle, + InProcessTransformResult result); + @Override - public CommittedResult handleResult( + public final CommittedResult handleResult( CommittedBundle inputBundle, InProcessTransformResult result) { - CommittedResult committedResult = - evaluationContext.handleResult(inputBundle, Collections.emptyList(), result); + CommittedResult committedResult = getCommittedResult(inputBundle, result); for (CommittedBundle outputBundle : committedResult.getOutputs()) { allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle)); } @@ -227,18 +231,33 @@ public CommittedResult handleResult( } @Override - public void handleThrowable(CommittedBundle inputBundle, Throwable t) { + public final void handleThrowable(CommittedBundle inputBundle, Throwable t) { allUpdates.offer(ExecutorUpdate.fromThrowable(t)); } } + /** + * The default {@link CompletionCallback}. The default completion callback is used to complete + * transform evaluations that are triggered due to the arrival of elements from an upstream + * transform, or for a source transform. + */ + private class DefaultCompletionCallback extends CompletionCallbackBase { + @Override + public CommittedResult getCommittedResult( + CommittedBundle inputBundle, InProcessTransformResult result) { + return evaluationContext.handleResult(inputBundle, + Collections.emptyList(), + result); + } + } + /** * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the * timers used to create the input to the {@link InProcessEvaluationContext evaluation context} * as part of the result. */ - private class TimerCompletionCallback implements CompletionCallback { + private class TimerCompletionCallback extends CompletionCallbackBase { private final Iterable timers; private TimerCompletionCallback(Iterable timers) { @@ -246,19 +265,9 @@ private TimerCompletionCallback(Iterable timers) { } @Override - public CommittedResult handleResult( + public CommittedResult getCommittedResult( CommittedBundle inputBundle, InProcessTransformResult result) { - CommittedResult committedResult = - evaluationContext.handleResult(inputBundle, timers, result); - for (CommittedBundle outputBundle : committedResult.getOutputs()) { - allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle)); - } - return committedResult; - } - - @Override - public void handleThrowable(CommittedBundle inputBundle, Throwable t) { - allUpdates.offer(ExecutorUpdate.fromThrowable(t)); + return evaluationContext.handleResult(inputBundle, timers, result); } } From eba8a49320f2ad9756e924fe36b6b4db4071bf8a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 13:11:12 -0700 Subject: [PATCH 19/29] Add TestFlinkPipelineRunner to FlinkRunnerRegistrar This makes the runner available for selection by integration tests. --- .../runners/flink/FlinkPipelineRunner.java | 16 +---- .../runners/flink/FlinkRunnerRegistrar.java | 4 +- .../flink/TestFlinkPipelineRunner.java | 66 +++++++++++++++++++ .../beam/runners/flink/FlinkTestPipeline.java | 2 +- 4 files changed, 71 insertions(+), 17 deletions(-) create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index a389d7a076c8..5e44d997e970 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -110,7 +110,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { this.flinkJobEnv.translate(pipeline); LOG.info("Starting execution of Flink program."); - + JobExecutionResult result; try { result = this.flinkJobEnv.executePipeline(); @@ -140,20 +140,6 @@ public FlinkPipelineOptions getPipelineOptions() { return options; } - /** - * Constructs a runner with default properties for testing. - * - * @return The newly created runner. - */ - public static FlinkPipelineRunner createForTest(boolean streaming) { - FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); - // we use [auto] for testing since this will make it pick up the Testing - // ExecutionEnvironment - options.setFlinkMaster("[auto]"); - options.setStreaming(streaming); - return new FlinkPipelineRunner(options); - } - @Override public Output apply( PTransform transform, Input input) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index cd99f4e65bce..ec61805a4ed0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -41,7 +41,9 @@ private FlinkRunnerRegistrar() { } public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of(FlinkPipelineRunner.class); + return ImmutableList.>>of( + FlinkPipelineRunner.class, + TestFlinkPipelineRunner.class); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java new file mode 100644 index 000000000000..24883c8035c2 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; + +public class TestFlinkPipelineRunner extends PipelineRunner { + + private FlinkPipelineRunner delegate; + + private TestFlinkPipelineRunner(FlinkPipelineOptions options) { + // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment + options.setFlinkMaster("[auto]"); + this.delegate = FlinkPipelineRunner.fromOptions(options); + } + + public static TestFlinkPipelineRunner fromOptions(PipelineOptions options) { + FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); + return new TestFlinkPipelineRunner(flinkOptions); + } + + public static TestFlinkPipelineRunner create(boolean streaming) { + FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class); + flinkOptions.setStreaming(streaming); + return TestFlinkPipelineRunner.fromOptions(flinkOptions); + } + + @Override + public + OutputT apply(PTransform transform, InputT input) { + return delegate.apply(transform, input); + } + + @Override + public FlinkRunnerResult run(Pipeline pipeline) { + return delegate.run(pipeline); + } + + public PipelineOptions getPipelineOptions() { + return delegate.getPipelineOptions(); + } +} + + diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java index f015a6680568..edde925c330c 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java @@ -60,7 +60,7 @@ public static FlinkTestPipeline createForStreaming() { * @return The Test Pipeline. */ private static FlinkTestPipeline create(boolean streaming) { - FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); + TestFlinkPipelineRunner flinkRunner = TestFlinkPipelineRunner.create(streaming); return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); } From 16a2c08a62b38b6b735e751ea31bbe60f0effba6 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 14:04:20 -0700 Subject: [PATCH 20/29] Configure RunnableOnService tests for Flink in batch mode Today Flink batch supports only global windows. This is a situation we intend our build to allow, eventually via JUnit category filtering. For now all the test classes that use non-global windows are excluded entirely via maven configuration. In the future, it should be on a per-test-method basis. --- runners/flink/runner/pom.xml | 105 ++++++++++++++++++++++++++--------- 1 file changed, 79 insertions(+), 26 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index a1d53703ad70..427e53b98ae5 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -34,31 +34,6 @@ jar - - - disable-runnable-on-service-tests - - true - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - runnable-on-service-tests - - true - - - - - - - - - @@ -87,6 +62,7 @@ flink-avro_2.10 ${flink.version} + org.apache.beam @@ -120,7 +96,23 @@ + + + + + org.apache.beam + java-sdk-all + tests + test + + + org.slf4j + slf4j-jdk14 + + + + org.apache.beam java-examples-all @@ -178,10 +170,71 @@ org.apache.maven.plugins maven-surefire-plugin + + + runnable-on-service-tests + integration-test + + test + + + org.apache.beam.sdk.testing.RunnableOnService + all + 4 + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=false" + ] + + + + + **/org/apache/beam/sdk/transforms/CombineTest.java + **/org/apache/beam/sdk/transforms/GroupByKeyTest.java + **/org/apache/beam/sdk/transforms/ViewTest.java + **/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java + **/org/apache/beam/sdk/transforms/windowing/WindowTest.java + **/org/apache/beam/sdk/transforms/windowing/WindowingTest.java + **/org/apache/beam/sdk/util/ReshuffleTest.java + + + + + streaming-runnable-on-service-tests + integration-test + + test + + + org.apache.beam.sdk.testing.RunnableOnService + all + 4 + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=true" + ] + + + + + + + - From c2314189c78d19d5f93a0a6cd86896290833c54e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:54:41 -0700 Subject: [PATCH 21/29] Remove unused threadCount from integration tests --- runners/flink/runner/pom.xml | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 427e53b98ae5..6fe9a91979af 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -179,8 +179,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all - 4 + none true org.apache.beam:java-sdk-all @@ -213,8 +212,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all - 4 + none true org.apache.beam:java-sdk-all From 4d9e4005bcae229f5ce023dbb1524e1f899cce4d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:55:16 -0700 Subject: [PATCH 22/29] Disable Flink streaming integration tests for now --- runners/flink/runner/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 6fe9a91979af..ce3fadd53014 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -211,6 +211,7 @@ test + true org.apache.beam.sdk.testing.RunnableOnService none true From e58c2159574a89535d6d27e26bb24339d3eac1ef Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 12:49:55 -0700 Subject: [PATCH 23/29] Special casing job exec AssertionError in TestFlinkPipelineRunner --- .../runners/flink/TestFlinkPipelineRunner.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java index 24883c8035c2..139aebf9dd2b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java @@ -26,6 +26,8 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.flink.runtime.client.JobExecutionException; + public class TestFlinkPipelineRunner extends PipelineRunner { private FlinkPipelineRunner delegate; @@ -55,7 +57,19 @@ OutputT apply(PTransform transform, InputT input) { @Override public FlinkRunnerResult run(Pipeline pipeline) { - return delegate.run(pipeline); + try { + return delegate.run(pipeline); + } catch (RuntimeException e) { + // Special case hack to pull out assertion errors from PAssert; instead there should + // probably be a better story along the lines of UserCodeException. + if (e.getCause() != null + && e.getCause() instanceof JobExecutionException + && e.getCause().getCause() instanceof AssertionError) { + throw (AssertionError) e.getCause().getCause(); + } else { + throw e; + } + } } public PipelineOptions getPipelineOptions() { From 97a388ac54a454279552551429582d9d174e9d7e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 09:38:55 +0200 Subject: [PATCH 24/29] Add hamcrest dependency to Flink Runner Without it the RunnableOnService tests seem to not work --- runners/flink/runner/pom.xml | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index ce3fadd53014..f5d05dfd78ff 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -99,6 +99,22 @@ + + org.hamcrest + hamcrest-all + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + org.apache.beam @@ -135,12 +151,6 @@ org.apache.flink flink-test-utils_2.10 ${flink.version} - test - - - org.mockito - mockito-all - test From de71ecf3957be94aa6b9faf07ba423bd94c3803c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 08:26:50 +0200 Subject: [PATCH 25/29] Fix Dangling Flink DataSets --- .../FlinkBatchPipelineTranslator.java | 14 ++++++++++++++ .../FlinkBatchTranslationContext.java | 18 +++++++++++++++++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 456cf09457b6..9d613552ee8d 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -24,7 +25,9 @@ import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.values.PValue; +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; @@ -52,6 +55,17 @@ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions op 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 // -------------------------------------------------------------------------------------------- diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index 71950cf216cb..501b1ea5555c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -43,6 +43,13 @@ public class FlinkBatchTranslationContext { private final Map> dataSets; private final Map, 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> danglingDataSets; + private final ExecutionEnvironment env; private final PipelineOptions options; @@ -55,10 +62,16 @@ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions op this.options = options; this.dataSets = new HashMap<>(); this.broadcastDataSets = new HashMap<>(); + + this.danglingDataSets = new HashMap<>(); } // ------------------------------------------------------------------------ - + + public Map> getDanglingDataSets() { + return danglingDataSets; + } + public ExecutionEnvironment getExecutionEnvironment() { return env; } @@ -69,12 +82,15 @@ public PipelineOptions getPipelineOptions() { @SuppressWarnings("unchecked") public DataSet getInputDataSet(PValue value) { + // assume that the DataSet is used as an input if retrieved here + danglingDataSets.remove(value); return (DataSet) dataSets.get(value); } public void setOutputDataSet(PValue value, DataSet set) { if (!dataSets.containsKey(value)) { dataSets.put(value, set); + danglingDataSets.put(value, set); } } From d16522b0d71e7d81f9f58be1ab57e5c17b2db0a7 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 13 May 2016 14:17:50 +0200 Subject: [PATCH 26/29] Fix faulty Flink Flatten when PCollectionList is empty --- .../FlinkBatchTransformTranslators.java | 32 +++++++++++++++---- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index a03352efae15..07785aa47c69 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Read; @@ -61,6 +62,7 @@ import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -78,6 +80,7 @@ import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,7 +94,7 @@ /** * Translators for transforming * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to - * Flink {@link org.apache.flink.api.java.DataSet}s + * Flink {@link org.apache.flink.api.java.DataSet}s. */ public class FlinkBatchTransformTranslators { @@ -465,15 +468,30 @@ public void translateNode(ParDo.BoundMulti transform, FlinkBatchTransla private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override + @SuppressWarnings("unchecked") public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { List> allInputs = context.getInput(transform).getAll(); DataSet result = null; - for(PCollection collection : allInputs) { - DataSet current = context.getInputDataSet(collection); - if (result == null) { - result = current; - } else { - result = result.union(current); + 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 dummySource = + context.getExecutionEnvironment().fromElements("dummy"); + result = dummySource.flatMap(new FlatMapFunction() { + @Override + public void flatMap(String s, Collector collector) throws Exception { + // never return anything + } + }).returns(new CoderTypeInformation<>((Coder) VoidCoder.of())); + } else { + for (PCollection collection : allInputs) { + DataSet current = context.getInputDataSet(collection); + if (result == null) { + result = current; + } else { + result = result.union(current); + } } } context.setOutputDataSet(context.getOutput(transform), result); From 7972dcdcaf084505823d8aa283c9cfe867bdc219 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 10 May 2016 13:53:03 +0200 Subject: [PATCH 27/29] [BEAM-270] Support Timestamps/Windows in Flink Batch With this change we always use WindowedValue for the underlying Flink DataSets instead of just T. This allows us to support windowing as well. This changes also a lot of other stuff enabled by the above: - Use WindowedValue throughout - Add proper translation for Window.into() - Make side inputs window aware - Make GroupByKey and Combine transformations window aware, this includes support for merging windows. GroupByKey is implemented as a Combine with a concatenating CombineFn, for simplicity This removes Flink specific transformations for things that are handled by builtin sources/sinks, among other things this: - Removes special translation for AvroIO.Read/Write and TextIO.Read/Write - Removes special support for Write.Bound, this was not working properly and is now handled by the Beam machinery that uses DoFns for this - Removes special translation for binary Co-Group, the code was still in there but was never used With this change all RunnableOnService tests run on Flink Batch. --- runners/flink/runner/pom.xml | 10 - .../FlinkBatchPipelineTranslator.java | 31 +- .../FlinkBatchTransformTranslators.java | 821 +++++++++++------- .../FlinkBatchTranslationContext.java | 56 +- .../FlinkStreamingTransformTranslators.java | 22 +- .../FlinkStreamingTranslationContext.java | 29 +- .../functions/FlinkAssignContext.java | 56 ++ .../functions/FlinkAssignWindows.java | 51 ++ .../FlinkCoGroupKeyedListAggregator.java | 61 -- .../functions/FlinkDoFnFunction.java | 194 ++--- .../FlinkKeyedListAggregationFunction.java | 78 -- .../FlinkMergingPartialReduceFunction.java | 211 +++++ .../functions/FlinkMergingReduceFunction.java | 215 +++++ .../FlinkMultiOutputDoFnFunction.java | 157 ++-- .../FlinkMultiOutputProcessContext.java | 176 ++++ .../FlinkMultiOutputPruningFunction.java | 25 +- .../FlinkNoElementAssignContext.java | 71 ++ .../functions/FlinkPartialReduceFunction.java | 175 +++- .../functions/FlinkProcessContext.java | 324 +++++++ .../functions/FlinkReduceFunction.java | 174 +++- .../functions/SideInputInitializer.java | 75 ++ .../translation/functions/UnionCoder.java | 152 ---- .../types/CoderTypeInformation.java | 12 +- .../types/CoderTypeSerializer.java | 14 +- .../translation/types/KvCoderComperator.java | 102 +-- .../types/KvCoderTypeInformation.java | 63 +- .../wrappers/CombineFnAggregatorWrapper.java | 94 -- .../SerializableFnAggregatorWrapper.java | 31 +- .../wrappers/SinkOutputFormat.java | 10 +- .../wrappers/SourceInputFormat.java | 18 +- .../apache/beam/runners/flink/AvroITCase.java | 36 +- .../runners/flink/MaybeEmptyTestITCase.java | 9 +- .../beam/sdk/transforms/join/UnionCoder.java | 2 +- 33 files changed, 2380 insertions(+), 1175 deletions(-) create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index f5d05dfd78ff..18e81d259b74 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -202,16 +202,6 @@ ] - - - **/org/apache/beam/sdk/transforms/CombineTest.java - **/org/apache/beam/sdk/transforms/GroupByKeyTest.java - **/org/apache/beam/sdk/transforms/ViewTest.java - **/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java - **/org/apache/beam/sdk/transforms/windowing/WindowTest.java - **/org/apache/beam/sdk/transforms/windowing/WindowingTest.java - **/org/apache/beam/sdk/util/ReshuffleTest.java - diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 9d613552ee8d..98229ae331f3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -32,8 +32,8 @@ import org.slf4j.LoggerFactory; /** - * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs. - * This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineTranslator} + * {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a + * Flink batch job. */ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator { @@ -77,10 +77,13 @@ public void enterCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == null) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { currentCompositeTransform = transform; - if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { + if (transform instanceof CoGroupByKey && + node.getInput().expand().size() != 2) { // we can only optimize CoGroupByKey for input size 2 currentCompositeTransform = null; } @@ -94,7 +97,9 @@ public void leaveCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == transform) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { LOG.info(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); applyBatchTransform(transform, node, translator); @@ -120,10 +125,13 @@ public void visitTransform(TransformTreeNode node) { // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator == null) { LOG.info(node.getTransform().getClass().toString()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + throw new UnsupportedOperationException( + "The transform " + transform + " is currently not supported."); } applyBatchTransform(transform, node, translator); } @@ -133,7 +141,10 @@ public void visitValue(PValue value, TransformTreeNode producer) { // do nothing here } - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + private > void applyBatchTransform( + PTransform transform, + TransformTreeNode node, + BatchTransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @@ -150,8 +161,8 @@ public void visitValue(PValue value, TransformTreeNode producer) { /** * A translator of a {@link PTransform}. */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); + public interface BatchTransformTranslator { + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } private static String genSpaces(int n) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index 07785aa47c69..fda4d29556f3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -17,23 +17,23 @@ */ package org.apache.beam.runners.flink.translation; -import org.apache.beam.runners.flink.io.ConsoleIO; -import org.apache.beam.runners.flink.translation.functions.FlinkCoGroupKeyedListAggregator; -import org.apache.beam.runners.flink.translation.functions.FlinkCreateFunction; +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.FlinkKeyedListAggregationFunction; +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.FlinkMultiOutputDoFnFunction; 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.UnionCoder; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.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.AvroIO; import org.apache.beam.sdk.io.BoundedSource; @@ -41,60 +41,63 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; +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.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGbkResultSchema; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.join.UnionCoder; +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.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 com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.operators.CoGroupOperator; import org.apache.flink.api.java.operators.DataSink; 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.MapOperator; import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; /** - * Translators for transforming - * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to - * Flink {@link org.apache.flink.api.java.DataSet}s. + * Translators for transforming {@link PTransform PTransforms} to + * Flink {@link DataSet DataSets}. */ public class FlinkBatchTransformTranslators { @@ -103,113 +106,90 @@ public class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); + private static final Map< + Class, + FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); - // register the known translators static { TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); - // we don't need this because we translate the Combine.PerKey directly - //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); - - TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); - // TODO we're currently ignoring windows here but that has to change in the future - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); + TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslatorBatch()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); - - TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch()); - - TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch()); - TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); - TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch()); - - TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch()); - TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch()); - - // Flink-specific - TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch()); - } - public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator(PTransform transform) { + public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator( + PTransform transform) { return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class ReadSourceTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override public void translateNode(Read.Bounded transform, FlinkBatchTranslationContext context) { String name = transform.getName(); BoundedSource source = transform.getSource(); PCollection output = context.getOutput(transform); - Coder coder = output.getCoder(); - TypeInformation typeInformation = context.getTypeInfo(output); + TypeInformation> typeInformation = context.getTypeInfo(output); - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), - new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); + DataSource> dataSource = new DataSource<>( + context.getExecutionEnvironment(), + new SourceInputFormat<>(source, context.getPipelineOptions()), + typeInformation, + name); context.setOutputDataSet(output, dataSource); } } - private static class AvroIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class); + private static class WriteSinkTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); + public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { String name = transform.getName(); -// Schema schema = transform.getSchema(); - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - - // This is super hacky, but unfortunately we cannot get the type otherwise - Class extractedAvroType; - try { - Field typeField = transform.getClass().getDeclaredField("type"); - typeField.setAccessible(true); - @SuppressWarnings("unchecked") - Class avroType = (Class) typeField.get(transform); - extractedAvroType = avroType; - } catch (NoSuchFieldException | IllegalAccessException e) { - // we know that the field is there and it is accessible - throw new RuntimeException("Could not access type from AvroIO.Bound", e); - } - - DataSource source = new DataSource<>(context.getExecutionEnvironment(), - new AvroInputFormat<>(new Path(path), extractedAvroType), - typeInformation, name); + PValue input = context.getInput(transform); + DataSet> inputDataSet = context.getInputDataSet(input); - context.setOutputDataSet(output, source); + inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())) + .name(name); } } - private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class AvroIOWriteTranslatorBatch implements + FlinkBatchPipelineTranslator.BatchTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class); + @Override - public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + AvroIO.Write.Bound transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); int numShards = transform.getNumShards(); String shardNameTemplate = transform.getShardNameTemplate(); // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", + LOG.warn( + "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + LOG.warn( + "Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", + shardNameTemplate); // This is super hacky, but unfortunately we cannot get the type otherwise Class extractedAvroType; @@ -224,8 +204,17 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation throw new RuntimeException("Could not access type from AvroIO.Bound", e); } - DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path - (filenamePrefix), extractedAvroType)); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(context.getInput(transform).getCoder())); + + + DataSink dataSink = valueStream.output( + new AvroOutputFormat<>(new Path(filenamePrefix), extractedAvroType)); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -233,37 +222,16 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation } } - private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class); - - @Override - public void translateNode(TextIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); - String name = transform.getName(); - - TextIO.CompressionType compressionType = transform.getCompressionType(); - boolean needsValidation = transform.needsValidation(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); - LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); - - context.setOutputDataSet(output, source); - } - } - - private static class TextIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class TextIOWriteTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); @Override - public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode( + TextIO.Write.Bound transform, + FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + DataSet> inputDataSet = context.getInputDataSet(input); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); @@ -272,12 +240,25 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation 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); + 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); - //inputDataSet.print(); - DataSink dataSink = inputDataSet.writeAsText(filenamePrefix); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(transform.getCoder())); + + DataSink dataSink = valueStream.writeAsText(filenamePrefix); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -285,148 +266,392 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation } } - private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { + private static class WindowBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + @Override - public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); - inputDataSet.printOnTaskManager(transform.getName()); + + TypeInformation> resultTypeInfo = + context.getTypeInfo(context.getOutput(transform)); + + DataSet> inputDataSet = context.getInputDataSet(input); + + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowFn windowFn = windowingStrategy.getWindowFn(); + + FlinkAssignWindows assignWindowsFunction = + new FlinkAssignWindows<>(windowFn); + + DataSet> resultDataSet = inputDataSet + .flatMap(assignWindowsFunction) + .name(context.getOutput(transform).getName()) + .returns(resultTypeInfo); + + context.setOutputDataSet(context.getOutput(transform), resultDataSet); } } - private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class GroupByKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { - String name = transform.getName(); - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + public void translateNode( + GroupByKey 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>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + + Combine.KeyedCombineFn, List> combineFn = + new Concatenate().asKeyedFn(); + + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + + Coder> 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>> kvCoderTypeInformation = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + inputCoder, + windowingStrategy.getWindowFn().windowCoder())); + + TypeInformation>>> partialReduceTypeInfo = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), + windowingStrategy.getWindowFn().windowCoder())); + + Grouping>> inputGrouping = + new UnsortedGrouping<>( + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, + kvCoderTypeInformation)); + + FlinkPartialReduceFunction, ?> partialReduceFunction; + FlinkReduceFunction, List, ?> reduceFunction; + + if (windowingStrategy.getWindowFn().isNonMerging()) { + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + Collections., 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."); + } + WindowingStrategy intervalStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkMergingPartialReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkMergingReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + } + + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); + + Grouping>>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format VO + GroupReduceOperator< + WindowedValue>>, WindowedValue>>> outputDataSet = + new GroupReduceOperator<>( + intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName()); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); - inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); } } /** - * Translates a GroupByKey while ignoring window assignments. Current ignores windows. + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

For internal use to translate {@link GroupByKey}. For a large {@link PCollection} this + * is expected to crash! + * + *

This is copied from the dataflow runner code. + * + * @param the type of elements to concatenate. */ - private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class Concatenate extends Combine.CombineFn, List> { + @Override + public List createAccumulator() { + return new ArrayList(); + } @Override - public void translateNode(GroupByKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); + public List addInput(List accumulator, T input) { + accumulator.add(input); + return accumulator; + } - TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = createAccumulator(); + for (List accumulator : accumulators) { + result.addAll(accumulator); + } + return result; + } - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + @Override + public List extractOutput(List accumulator) { + return accumulator; + } - GroupReduceOperator, KV>> outputDataSet = - new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } - context.setOutputDataSet(context.getOutput(transform), outputDataSet); + @Override + public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); } } - private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + private static class CombinePerKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + Combine.PerKey> { @Override - public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + @SuppressWarnings("unchecked") + public void translateNode( + Combine.PerKey transform, + FlinkBatchTranslationContext context) { + DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - @SuppressWarnings("unchecked") - Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); + CombineFnBase.PerKeyCombineFn combineFn = + (CombineFnBase.PerKeyCombineFn) transform.getFn(); + + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); - KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + Coder accumulatorCoder; - Coder accumulatorCoder = - null; try { - accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + accumulatorCoder = + combineFn.getAccumulatorCoder( + context.getInput(transform).getPipeline().getCoderRegistry(), + inputCoder.getKeyCoder(), + inputCoder.getValueCoder()); } catch (CannotProvideCoderException e) { - e.printStackTrace(); - // TODO + throw new RuntimeException(e); } - TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); - TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + + TypeInformation>> kvCoderTypeInformation = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + inputCoder, + windowingStrategy.getWindowFn().windowCoder())); + + TypeInformation>> partialReduceTypeInfo = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), + windowingStrategy.getWindowFn().windowCoder())); + + Grouping>> inputGrouping = + new UnsortedGrouping<>( + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, + kvCoderTypeInformation)); + + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: transform.getSideInputs()) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } - Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + FlinkPartialReduceFunction partialReduceFunction; + FlinkReduceFunction reduceFunction; - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + if (windowingStrategy.getWindowFn().isNonMerging()) { + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; - // Partially GroupReduce the values into the intermediate format VA (combine) - GroupCombineOperator, KV> groupCombine = - new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, - "GroupCombine: " + transform.getName()); + partialReduceFunction = new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + reduceFunction = new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + } else { + if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) { + throw new UnsupportedOperationException( + "Merging WindowFn with windows other than IntervalWindow are not supported."); + } + WindowingStrategy intervalStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkMergingPartialReduceFunction<>( + combineFn, + intervalStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + reduceFunction = new FlinkMergingReduceFunction<>( + combineFn, + intervalStrategy, + sideInputStrategies, + context.getPipelineOptions()); + } - // Reduce fully to VO - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); - TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); + transformSideInputs(transform.getSideInputs(), groupCombine, context); - Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + TypeInformation>> reduceTypeInfo = + context.getTypeInfo(context.getOutput(transform)); - // Fully reduce the values and create output format VO - GroupReduceOperator, KV> outputDataSet = - new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + Grouping>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format OutputT + GroupReduceOperator< + WindowedValue>, WindowedValue>> outputDataSet = + new GroupReduceOperator<>( + intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + + transformSideInputs(transform.getSideInputs(), outputDataSet, context); context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } -// private static class CombineGroupedValuesTranslator implements FlinkPipelineTranslator.TransformTranslator> { -// -// @Override -// public void translateNode(Combine.GroupedValues transform, TranslationContext context) { -// DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); -// -// Combine.KeyedCombineFn keyedCombineFn = transform.getFn(); -// -// GroupReduceFunction, KV> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn); -// -// TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); -// -// Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType())); -// -// GroupReduceOperator, KV> outputDataSet = -// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); -// context.setOutputDataSet(transform.getOutput(), outputDataSet); -// } -// } - - private static class ParDoBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class); + private static class ParDoBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + ParDo.Bound> { @Override - public void translateNode(ParDo.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + ParDo.Bound transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - final DoFn doFn = transform.getFn(); + final DoFn doFn = transform.getFn(); - TypeInformation typeInformation = context.getTypeInfo(context.getOutput(transform)); + TypeInformation> typeInformation = + context.getTypeInfo(context.getOutput(transform)); - FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + List> sideInputs = transform.getSideInputs(); - transformSideInputs(transform.getSideInputs(), outputDataSet, context); + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: sideInputs) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } + + FlinkDoFnFunction doFnWrapper = + new FlinkDoFnFunction<>( + doFn, + context.getOutput(transform).getWindowingStrategy(), + sideInputStrategies, + context.getPipelineOptions()); + + MapPartitionOperator, WindowedValue> outputDataSet = + new MapPartitionOperator<>( + inputDataSet, + typeInformation, + doFnWrapper, + transform.getName()); + + transformSideInputs(sideInputs, outputDataSet, context); context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } - private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class); + private static class ParDoBoundMultiTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + ParDo.BoundMulti> { @Override - public void translateNode(ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + ParDo.BoundMulti transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - final DoFn doFn = transform.getFn(); + final DoFn doFn = transform.getFn(); Map, PCollection> outputs = context.getOutput(transform).getAll(); Map, Integer> outputMap = Maps.newHashMap(); - // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this + // put the main output at index 0, FlinkMultiOutputDoFnFunction expects this outputMap.put(transform.getMainOutputTag(), 0); int count = 1; for (TupleTag tag: outputs.keySet()) { @@ -435,58 +660,118 @@ public void translateNode(ParDo.BoundMulti transform, FlinkBatchTransla } } + // 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> outputCoders = Lists.newArrayList(); for (PCollection coll: outputs.values()) { outputCoders.add(coll.getCoder()); + windowingStrategy = coll.getWindowingStrategy(); + } + + if (windowingStrategy == null) { + throw new IllegalStateException("No outputs defined."); } UnionCoder unionCoder = UnionCoder.of(outputCoders); - @SuppressWarnings("unchecked") - TypeInformation typeInformation = new CoderTypeInformation<>(unionCoder); + TypeInformation> typeInformation = + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + unionCoder, + windowingStrategy.getWindowFn().windowCoder())); - @SuppressWarnings("unchecked") - FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + List> sideInputs = transform.getSideInputs(); - transformSideInputs(transform.getSideInputs(), outputDataSet, context); + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: sideInputs) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } - for (Map.Entry, PCollection> output: outputs.entrySet()) { - TypeInformation outputType = context.getTypeInfo(output.getValue()); - int outputTag = outputMap.get(output.getKey()); - FlinkMultiOutputPruningFunction pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag); - FlatMapOperator pruningOperator = new - FlatMapOperator<>(outputDataSet, outputType, - pruningFunction, output.getValue().getName()); - context.setOutputDataSet(output.getValue(), pruningOperator); + @SuppressWarnings("unchecked") + FlinkMultiOutputDoFnFunction doFnWrapper = + new FlinkMultiOutputDoFnFunction( + doFn, + windowingStrategy, + sideInputStrategies, + context.getPipelineOptions(), + outputMap); + + MapPartitionOperator, WindowedValue> taggedDataSet = + new MapPartitionOperator<>( + inputDataSet, + typeInformation, + doFnWrapper, + transform.getName()); + + transformSideInputs(sideInputs, taggedDataSet, context); + for (Map.Entry, PCollection> output: outputs.entrySet()) { + pruneOutput( + taggedDataSet, + context, + outputMap.get(output.getKey()), + (PCollection) output.getValue()); } } + + private void pruneOutput( + MapPartitionOperator, WindowedValue> taggedDataSet, + FlinkBatchTranslationContext context, + int integerTag, + PCollection collection) { + TypeInformation> outputType = context.getTypeInfo(collection); + + FlinkMultiOutputPruningFunction pruningFunction = + new FlinkMultiOutputPruningFunction<>(integerTag); + + FlatMapOperator, WindowedValue> pruningOperator = + new FlatMapOperator<>( + taggedDataSet, + outputType, + pruningFunction, + collection.getName()); + + context.setOutputDataSet(collection, pruningOperator); + } } - private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class FlattenPCollectionTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + Flatten.FlattenPCollectionList> { @Override @SuppressWarnings("unchecked") - public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { + public void translateNode( + Flatten.FlattenPCollectionList transform, + FlinkBatchTranslationContext context) { + List> allInputs = context.getInput(transform).getAll(); - DataSet result = null; + DataSet> 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 dummySource = context.getExecutionEnvironment().fromElements("dummy"); - result = dummySource.flatMap(new FlatMapFunction() { + result = dummySource.flatMap(new FlatMapFunction>() { @Override - public void flatMap(String s, Collector collector) throws Exception { + public void flatMap(String s, Collector> collector) throws Exception { // never return anything } - }).returns(new CoderTypeInformation<>((Coder) VoidCoder.of())); + }).returns( + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + (Coder) VoidCoder.of(), + GlobalWindow.Coder.INSTANCE))); } else { for (PCollection collection : allInputs) { - DataSet current = context.getInputDataSet(collection); + DataSet> current = context.getInputDataSet(collection); if (result == null) { result = current; } else { @@ -494,103 +779,47 @@ public void flatMap(String s, Collector collector) throws Exception { } } } - context.setOutputDataSet(context.getOutput(transform), result); - } - } - private static class CreatePCollectionViewTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(View.CreatePCollectionView transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - PCollectionView input = transform.apply(null); - context.setSideInputDataSet(input, inputDataSet); + // 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>() { + @Override + public boolean filter(WindowedValue tWindowedValue) throws Exception { + return true; + } + }).name("UnionFixFilter"); + context.setOutputDataSet(context.getOutput(transform), result); } } - private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class CreatePCollectionViewTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + View.CreatePCollectionView> { @Override - public void translateNode(Create.Values transform, FlinkBatchTranslationContext context) { - TypeInformation typeInformation = context.getOutputTypeInfo(); - Iterable elements = transform.getElements(); - - // we need to serialize the elements to byte arrays, since they might contain - // elements that are not serializable by Java serialization. We deserialize them - // in the FlatMap function using the Coder. - - List serializedElements = Lists.newArrayList(); - Coder coder = context.getOutput(transform).getCoder(); - for (OUT element: elements) { - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - try { - coder.encode(element, bao, Coder.Context.OUTER); - serializedElements.add(bao.toByteArray()); - } catch (IOException e) { - throw new RuntimeException("Could not serialize Create elements using Coder: " + e); - } - } + public void translateNode( + View.CreatePCollectionView transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); - FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); - FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); + PCollectionView input = transform.getView(); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); + context.setSideInputDataSet(input, inputDataSet); } } - private static void transformSideInputs(List> sideInputs, - MapPartitionOperator outputDataSet, - FlinkBatchTranslationContext context) { + private static void transformSideInputs( + List> sideInputs, + SingleInputUdfOperator outputDataSet, + FlinkBatchTranslationContext context) { // get corresponding Flink broadcast DataSets - for(PCollectionView input : sideInputs) { + for (PCollectionView input : sideInputs) { DataSet broadcastSet = context.getSideInputDataSet(input); outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); } } -// Disabled because it depends on a pending pull request to the DataFlowSDK - /** - * Special composite transform translator. Only called if the CoGroup is two dimensional. - * @param - */ - private static class CoGroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @Override - public void translateNode(CoGroupByKey transform, FlinkBatchTranslationContext context) { - KeyedPCollectionTuple input = context.getInput(transform); - - CoGbkResultSchema schema = input.getCoGbkResultSchema(); - List> keyedCollections = input.getKeyedCollections(); - - KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection1 = keyedCollections.get(0); - KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection2 = keyedCollections.get(1); - - TupleTag tupleTag1 = taggedCollection1.getTupleTag(); - TupleTag tupleTag2 = taggedCollection2.getTupleTag(); - - PCollection> collection1 = taggedCollection1.getCollection(); - PCollection> collection2 = taggedCollection2.getCollection(); - - DataSet> inputDataSet1 = context.getInputDataSet(collection1); - DataSet> inputDataSet2 = context.getInputDataSet(collection2); - - TypeInformation> typeInfo = context.getOutputTypeInfo(); - - FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); - - Keys.ExpressionKeys> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); - Keys.ExpressionKeys> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); - - DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, - keySelector1, keySelector2, - aggregator, typeInfo, null, transform.getName()); - context.setOutputDataSet(context.getOutput(transform), out); - } - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - private FlinkBatchTransformTranslators() {} } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index 501b1ea5555c..ecc3a65c7965 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -18,26 +18,28 @@ package org.apache.beam.runners.flink.translation; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TypedPValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; import java.util.HashMap; import java.util.Map; +/** + * Helper for {@link FlinkBatchPipelineTranslator} and translators in + * {@link FlinkBatchTransformTranslators}. + */ public class FlinkBatchTranslationContext { private final Map> dataSets; @@ -81,13 +83,13 @@ public PipelineOptions getPipelineOptions() { } @SuppressWarnings("unchecked") - public DataSet getInputDataSet(PValue value) { + public DataSet> getInputDataSet(PValue value) { // assume that the DataSet is used as an input if retrieved here danglingDataSets.remove(value); - return (DataSet) dataSets.get(value); + return (DataSet>) dataSets.get(value); } - public void setOutputDataSet(PValue value, DataSet set) { + public void setOutputDataSet(PValue value, DataSet> set) { if (!dataSets.containsKey(value)) { dataSets.put(value, set); danglingDataSets.put(value, set); @@ -107,40 +109,32 @@ public DataSet getSideInputDataSet(PCollectionView value) { return (DataSet) broadcastDataSets.get(value); } - public void setSideInputDataSet(PCollectionView value, DataSet set) { + public void setSideInputDataSet( + PCollectionView value, + DataSet> set) { if (!broadcastDataSets.containsKey(value)) { broadcastDataSets.put(value, set); } } - - @SuppressWarnings("unchecked") - public TypeInformation getTypeInfo(PInput output) { - if (output instanceof TypedPValue) { - Coder outputCoder = ((TypedPValue) output).getCoder(); - if (outputCoder instanceof KvCoder) { - return new KvCoderTypeInformation((KvCoder) outputCoder); - } else { - return new CoderTypeInformation(outputCoder); - } - } - return new GenericTypeInfo<>((Class)Object.class); - } - - public TypeInformation getInputTypeInfo() { - return getTypeInfo(currentTransform.getInput()); - } - public TypeInformation getOutputTypeInfo() { - return getTypeInfo((PValue) currentTransform.getOutput()); + @SuppressWarnings("unchecked") + public TypeInformation> getTypeInfo(PCollection collection) { + Coder valueCoder = collection.getCoder(); + WindowedValue.FullWindowedValueCoder windowedValueCoder = + WindowedValue.getFullCoder( + valueCoder, + collection.getWindowingStrategy().getWindowFn().windowCoder()); + + return new CoderTypeInformation<>(windowedValueCoder); } @SuppressWarnings("unchecked") - I getInput(PTransform transform) { - return (I) currentTransform.getInput(); + T getInput(PTransform transform) { + return (T) currentTransform.getInput(); } @SuppressWarnings("unchecked") - O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); + T getOutput(PTransform transform) { + return (T) currentTransform.getOutput(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 618727dae4d9..7aedf5520502 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation; -import org.apache.beam.runners.flink.translation.functions.UnionCoder; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.FlinkCoder; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; @@ -45,6 +44,7 @@ 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.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -228,29 +228,15 @@ public void translateNode(Read.Bounded transform, FlinkStreamingTranslationCo BoundedSource boundedSource = transform.getSource(); PCollection output = context.getOutput(transform); - Coder defaultOutputCoder = boundedSource.getDefaultOutputCoder(); - CoderTypeInformation typeInfo = new CoderTypeInformation<>(defaultOutputCoder); + TypeInformation> typeInfo = context.getTypeInfo(output); - DataStream source = context.getExecutionEnvironment().createInput( + DataStream> source = context.getExecutionEnvironment().createInput( new SourceInputFormat<>( boundedSource, context.getPipelineOptions()), typeInfo); - DataStream> windowedStream = source.flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(T value, Collector> out) throws Exception { - out.collect( - WindowedValue.of(value, - Instant.now(), - GlobalWindow.INSTANCE, - PaneInfo.NO_FIRING)); - } - }) - .assignTimestampsAndWatermarks(new IngestionTimeExtractor>()); - - context.setOutputDataStream(output, windowedStream); + context.setOutputDataStream(output, source); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java index 8bc73172405c..0cb80baa7cc8 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java @@ -17,21 +17,30 @@ */ package org.apache.beam.runners.flink.translation; +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 com.google.common.base.Preconditions; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import java.util.HashMap; import java.util.Map; +/** + * Helper for keeping track of which {@link DataStream DataStreams} map + * to which {@link PTransform PTransforms}. + */ public class FlinkStreamingTranslationContext { private final StreamExecutionEnvironment env; @@ -80,12 +89,24 @@ public void setCurrentTransform(AppliedPTransform currentTransform) { } @SuppressWarnings("unchecked") - public I getInput(PTransform transform) { - return (I) currentTransform.getInput(); + public TypeInformation> getTypeInfo(PCollection collection) { + Coder valueCoder = collection.getCoder(); + WindowedValue.FullWindowedValueCoder windowedValueCoder = + WindowedValue.getFullCoder( + valueCoder, + collection.getWindowingStrategy().getWindowFn().windowCoder()); + + return new CoderTypeInformation<>(windowedValueCoder); + } + + + @SuppressWarnings("unchecked") + public T getInput(PTransform transform) { + return (T) currentTransform.getInput(); } @SuppressWarnings("unchecked") - public O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); + public T getOutput(PTransform transform) { + return (T) currentTransform.getOutput(); } } 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 new file mode 100644 index 000000000000..7ea8c202f9d5 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.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.functions; + +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; + +import java.util.Collection; + +/** + * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for + * Flink functions. + */ +class FlinkAssignContext + extends WindowFn.AssignContext { + private final WindowedValue value; + + FlinkAssignContext(WindowFn fn, WindowedValue value) { + fn.super(); + this.value = value; + } + + @Override + public InputT element() { + return value.getValue(); + } + + @Override + public Instant timestamp() { + return value.getTimestamp(); + } + + @Override + public Collection windows() { + return value.getWindows(); + } + +} 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 new file mode 100644 index 000000000000..e07e49a2f060 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.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.functions; + +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; + +import java.util.Collection; + +/** + * Flink {@link FlatMapFunction} for implementing + * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound}. + */ +public class FlinkAssignWindows + implements FlatMapFunction, WindowedValue> { + + private final WindowFn windowFn; + + public FlinkAssignWindows(WindowFn windowFn) { + this.windowFn = windowFn; + } + + @Override + public void flatMap( + WindowedValue input, Collector> collector) throws Exception { + Collection windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input)); + for (W window: windows) { + collector.collect( + WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane())); + } + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java deleted file mode 100644 index 8e7cdd75ca48..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGbkResultSchema; -import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TupleTag; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.util.Collector; - -import java.util.ArrayList; -import java.util.List; - - -public class FlinkCoGroupKeyedListAggregator implements CoGroupFunction, KV, KV>{ - - private CoGbkResultSchema schema; - private TupleTag tupleTag1; - private TupleTag tupleTag2; - - public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag tupleTag1, TupleTag tupleTag2) { - this.schema = schema; - this.tupleTag1 = tupleTag1; - this.tupleTag2 = tupleTag2; - } - - @Override - public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { - K k = null; - List result = new ArrayList<>(); - int index1 = schema.getIndex(tupleTag1); - for (KV entry : first) { - k = entry.getKey(); - result.add(new RawUnionValue(index1, entry.getValue())); - } - int index2 = schema.getIndex(tupleTag2); - for (KV entry : second) { - k = entry.getKey(); - result.add(new RawUnionValue(index2, entry.getValue())); - } - out.collect(KV.of(k, new CoGbkResult(schema, result))); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 3566f7e1070e..89243a3ede28 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -18,173 +18,85 @@ package org.apache.beam.runners.flink.translation.functions; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; -import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.util.Collector; -import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; +import java.util.Map; /** * Encapsulates a {@link org.apache.beam.sdk.transforms.DoFn} * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. */ -public class FlinkDoFnFunction extends RichMapPartitionFunction { +public class FlinkDoFnFunction + extends RichMapPartitionFunction, WindowedValue> { - private final DoFn doFn; + private final DoFn doFn; private final SerializedPipelineOptions serializedOptions; - public FlinkDoFnFunction(DoFn doFn, PipelineOptions options) { - this.doFn = doFn; - this.serializedOptions = new SerializedPipelineOptions(options); - } - - @Override - public void mapPartition(Iterable values, Collector out) throws Exception { - ProcessContext context = new ProcessContext(doFn, out); - this.doFn.startBundle(context); - for (IN value : values) { - context.inValue = value; - doFn.processElement(context); - } - this.doFn.finishBundle(context); - } - - private class ProcessContext extends DoFn.ProcessContext { - - IN inValue; - Collector outCollector; - - public ProcessContext(DoFn fn, Collector outCollector) { - fn.super(); - super.setupDelegateAggregators(); - this.outCollector = outCollector; - } - - @Override - public IN element() { - return this.inValue; - } - + private final Map, WindowingStrategy> sideInputs; - @Override - public Instant timestamp() { - return Instant.now(); - } + private final boolean requiresWindowAccess; + private final boolean hasSideInputs; - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + private final WindowingStrategy windowingStrategy; - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - @Override - public StateInternals stateInternals() { - return null; - } - - @Override - public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { - - } - - @Override - public TimerInternals timerInternals() { - return null; - } + public FlinkDoFnFunction( + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions options) { + this.doFn = doFn; + this.sideInputs = sideInputs; + this.serializedOptions = new SerializedPipelineOptions(options); + this.windowingStrategy = windowingStrategy; - @Override - public Collection windows() { - return ImmutableList.of(GlobalWindow.INSTANCE); - } + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.hasSideInputs = !sideInputs.isEmpty(); + } - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + @Override + public void mapPartition( + Iterable> values, + Collector> out) throws Exception { + + FlinkProcessContext context = new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); - @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - } + this.doFn.startBundle(context); - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - throw new RuntimeException("sideInput() not implemented."); + if (!requiresWindowAccess || hasSideInputs) { + // we don't need to explode the windows + for (WindowedValue value : values) { + context = context.forWindowedValue(value); + doFn.processElement(context); + } + } else { + // we need to explode the windows because we have per-window + // side inputs and window access also only works if an element + // is in only one window + for (WindowedValue value : values) { + for (WindowedValue explodedValue: value.explodeWindows()) { + context = context.forWindowedValue(value); + doFn.processElement(context); } - }; - } - - @Override - public PipelineOptions getPipelineOptions() { - return serializedOptions.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId()); - List> windowedValueList = new ArrayList<>(sideInput.size()); - for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); } - return view.fromIterableInternal(windowedValueList); } - @Override - public void output(OUT output) { - outCollector.collect(output); - } - - @Override - public void outputWithTimestamp(OUT output, Instant timestamp) { - // not FLink's way, just output normally - output(output); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - // ignore the side output, this can happen when a user does not register - // side outputs but then outputs using a freshly created TupleTag. - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutput(tag, output); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } - - + // set the windowed value to null so that the logic + // or outputting in finishBundle kicks in + context = context.forWindowedValue(null); + this.doFn.finishBundle(context); } + } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java deleted file mode 100644 index 7c7084db287c..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - -import org.apache.beam.sdk.values.KV; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.util.Collector; - -import java.util.Iterator; - -/** - * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a - * {@link org.apache.beam.sdk.transforms.GroupByKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and collects - * the values in a {@code List}. - */ -public class FlinkKeyedListAggregationFunction implements GroupReduceFunction, KV>> { - - @Override - public void reduce(Iterable> values, Collector>> out) throws Exception { - Iterator> it = values.iterator(); - KV first = it.next(); - Iterable passThrough = new PassThroughIterable<>(first, it); - out.collect(KV.of(first.getKey(), passThrough)); - } - - private static class PassThroughIterable implements Iterable, Iterator { - private KV first; - private Iterator> iterator; - - public PassThroughIterable(KV first, Iterator> iterator) { - this.first = first; - this.iterator = iterator; - } - - @Override - public Iterator iterator() { - return this; - } - - @Override - public boolean hasNext() { - return first != null || iterator.hasNext(); - } - - @Override - public V next() { - if (first != null) { - V result = first.getValue(); - first = null; - return result; - } else { - return iterator.next().getValue(); - } - } - - @Override - public void remove() { - throw new UnsupportedOperationException("Cannot remove elements from input."); - } - } -} 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 new file mode 100644 index 000000000000..47f920770d45 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.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.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +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 com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * 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 + extends FlinkPartialReduceFunction { + + public FlinkMergingPartialReduceFunction( + CombineFnBase.PerKeyCombineFn combineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + super(combineFn, windowingStrategy, sideInputs, pipelineOptions); + } + + @Override + public void combine( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) 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>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + System.out.println("ADDING EXLODE: " + exploded); + sortedInput.add(exploded); + } + } + System.out.println("SORTED SIZE: " + sortedInput.size()); + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> 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>> iterator = sortedInput.iterator(); + + // create accumulator using the first elements key + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (currentWindow.equals(nextWindow)) { + // continue accumulating and merge windows + + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + 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; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + + // we have to keep track so that we can set the context to the right + // windowed value when windows change in the iterable + currentValue = nextValue; + } + + // 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>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> 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> 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> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } +} 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 new file mode 100644 index 000000000000..3bd08075401f --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.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.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +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 com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * 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 + extends FlinkReduceFunction { + + public FlinkMergingReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + super(keyedCombineFn, windowingStrategy, sideInputs, pipelineOptions); + } + + @Override + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) 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>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> 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>> iterator = sortedInput.iterator(); + + // get the first accumulator + WindowedValue> 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 windowTimestamps = new ArrayList<>(); + windowTimestamps.add(currentValue.getTimestamp()); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating and merge windows + + processContext = processContext.forWindowedValue(nextValue); + + accumulator = combineFnRunner.mergeAccumulators( + key, ImmutableList.of(accumulator, nextValue.getValue().getValue()), processContext); + + windowTimestamps.add(nextValue.getTimestamp()); + } else { + // emit the value that we currently have + processContext = processContext.forWindowedValue(currentValue); + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); + + windowTimestamps.clear(); + + currentWindow = nextWindow; + accumulator = nextValue.getValue().getValue(); + windowTimestamps.add(nextValue.getTimestamp()); + } + + // we have to keep track so that we can set the context to the right + // windowed value when windows change in the iterable + currentValue = nextValue; + } + + // if at the end of the iteration we have a change in windows + // the ProcessContext will not have been updated + processContext = processContext.forWindowedValue(currentValue); + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + 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>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> 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> 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> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java index 476dc5e5f8e5..f92e76fa60cb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -18,28 +18,17 @@ package org.apache.beam.runners.flink.translation.functions; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableList; - import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.util.Collector; -import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.List; import java.util.Map; /** @@ -50,112 +39,72 @@ * 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 FlinkMultiOutputDoFnFunction extends RichMapPartitionFunction { - - private final DoFn doFn; - private final SerializedPipelineOptions serializedPipelineOptions; - private final Map, Integer> outputMap; - - public FlinkMultiOutputDoFnFunction(DoFn doFn, PipelineOptions options, Map, Integer> outputMap) { - this.doFn = doFn; - this.serializedPipelineOptions = new SerializedPipelineOptions(options); - this.outputMap = outputMap; - } - - @Override - public void mapPartition(Iterable values, Collector out) throws Exception { - ProcessContext context = new ProcessContext(doFn, out); - this.doFn.startBundle(context); - for (IN value : values) { - context.inValue = value; - doFn.processElement(context); - } - this.doFn.finishBundle(context); - } +public class FlinkMultiOutputDoFnFunction + extends RichMapPartitionFunction, WindowedValue> { - private class ProcessContext extends DoFn.ProcessContext { + private final DoFn doFn; + private final SerializedPipelineOptions serializedOptions; - IN inValue; - Collector outCollector; + private final Map, Integer> outputMap; - public ProcessContext(DoFn fn, Collector outCollector) { - fn.super(); - this.outCollector = outCollector; - } + private final Map, WindowingStrategy> sideInputs; - @Override - public IN element() { - return this.inValue; - } + private final boolean requiresWindowAccess; + private final boolean hasSideInputs; - @Override - public Instant timestamp() { - return Instant.now(); - } + private final WindowingStrategy windowingStrategy; - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } + public FlinkMultiOutputDoFnFunction( + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions options, + Map, Integer> outputMap) { + this.doFn = doFn; + this.serializedOptions = new SerializedPipelineOptions(options); + this.outputMap = outputMap; - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.hasSideInputs = !sideInputs.isEmpty(); + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + } - @Override - public WindowingInternals windowingInternals() { - return null; - } + @Override + public void mapPartition( + Iterable> values, + Collector> out) throws Exception { + + FlinkProcessContext context = new FlinkMultiOutputProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + outputMap, + sideInputs); - @Override - public PipelineOptions getPipelineOptions() { - return serializedPipelineOptions.getPipelineOptions(); - } + this.doFn.startBundle(context); - @Override - public T sideInput(PCollectionView view) { - List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal() - .getId()); - List> windowedValueList = new ArrayList<>(sideInput.size()); - for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); + if (!requiresWindowAccess || hasSideInputs) { + // we don't need to explode the windows + for (WindowedValue value : values) { + context = context.forWindowedValue(value); + doFn.processElement(context); } - return view.fromIterableInternal(windowedValueList); - } - - @Override - public void output(OUT value) { - // assume that index 0 is the default output - outCollector.collect(new RawUnionValue(0, value)); - } - - @Override - public void outputWithTimestamp(OUT output, Instant timestamp) { - // not FLink's way, just output normally - output(output); - } - - @Override - @SuppressWarnings("unchecked") - public void sideOutput(TupleTag tag, T value) { - Integer index = outputMap.get(tag); - if (index != null) { - outCollector.collect(new RawUnionValue(index, value)); + } else { + // we need to explode the windows because we have per-window + // side inputs and window access also only works if an element + // is in only one window + for (WindowedValue value : values) { + for (WindowedValue explodedValue: value.explodeWindows()) { + context = context.forWindowedValue(value); + doFn.processElement(context); + } } } - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutput(tag, output); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return null; - } + this.doFn.finishBundle(context); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java new file mode 100644 index 000000000000..71b6d27ddba0 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.options.PipelineOptions; +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.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.api.common.functions.RuntimeContext; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collection; +import java.util.Map; + +/** + * {@link DoFn.ProcessContext} for {@link FlinkMultiOutputDoFnFunction} that supports + * side outputs. + */ +class FlinkMultiOutputProcessContext + extends FlinkProcessContext { + + // we need a different Collector from the base class + private final Collector> collector; + + private final Map, Integer> outputMap; + + + FlinkMultiOutputProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Collector> collector, + Map, Integer> outputMap, + Map, WindowingStrategy> sideInputs) { + super( + pipelineOptions, + runtimeContext, + doFn, + windowingStrategy, + new Collector>() { + @Override + public void collect(WindowedValue outputTWindowedValue) { + + } + + @Override + public void close() { + + } + }, + sideInputs); + + this.collector = collector; + this.outputMap = outputMap; + } + + @Override + public FlinkProcessContext forWindowedValue( + WindowedValue windowedValue) { + this.windowedValue = windowedValue; + return this; + } + + @Override + public void outputWithTimestamp(OutputT value, Instant timestamp) { + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPane())); + } + } + + @Override + protected void outputWithTimestampAndWindow( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), timestamp, windows, pane)); + } + + @Override + @SuppressWarnings("unchecked") + public void sideOutput(TupleTag tag, T value) { + if (windowedValue != null) { + sideOutputWithTimestamp(tag, value, windowedValue.getTimestamp()); + } else { + sideOutputWithTimestamp(tag, value, null); + } + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T value, Instant timestamp) { + Integer index = outputMap.get(tag); + + if (index == null) { + throw new IllegalArgumentException("Unknown side output tag: " + tag); + } + + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + new RawUnionValue(index, value), + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + new RawUnionValue(index, value), + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPane())); + } + + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 58a36b27c5dd..9205a5520f82 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -18,27 +18,34 @@ 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 FlatMap function that filters out those elements that don't belong in this output. We need - * this to implement MultiOutput ParDo functions. + * 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 FlinkMultiOutputDoFnFunction}. */ -public class FlinkMultiOutputPruningFunction implements FlatMapFunction { +public class FlinkMultiOutputPruningFunction + implements FlatMapFunction, WindowedValue> { - private final int outputTag; + private final int ourOutputTag; - public FlinkMultiOutputPruningFunction(int outputTag) { - this.outputTag = outputTag; + public FlinkMultiOutputPruningFunction(int ourOutputTag) { + this.ourOutputTag = ourOutputTag; } @Override @SuppressWarnings("unchecked") - public void flatMap(RawUnionValue rawUnionValue, Collector collector) throws Exception { - if (rawUnionValue.getUnionTag() == outputTag) { - collector.collect((T) rawUnionValue.getValue()); + public void flatMap( + WindowedValue windowedValue, + Collector> collector) throws Exception { + int unionTag = windowedValue.getValue().getUnionTag(); + if (unionTag == ourOutputTag) { + collector.collect( + (WindowedValue) windowedValue.withValue(windowedValue.getValue().getValue())); } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java new file mode 100644 index 000000000000..892f7a1f33f0 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; + +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * {@link WindowFn.AssignContext} for calling a {@link WindowFn} for elements emitted from + * {@link org.apache.beam.sdk.transforms.DoFn#startBundle(DoFn.Context)} + * or {@link DoFn#finishBundle(DoFn.Context)}. + * + *

In those cases the {@code WindowFn} is not allowed to access any element information. + */ +class FlinkNoElementAssignContext + extends WindowFn.AssignContext { + + private final InputT element; + private final Instant timestamp; + + FlinkNoElementAssignContext( + WindowFn fn, + InputT element, + Instant timestamp) { + fn.super(); + + this.element = element; + // the timestamp can be null, in that case output is called + // without a timestamp + this.timestamp = timestamp; + } + + @Override + public InputT element() { + return element; + } + + @Override + public Instant timestamp() { + if (timestamp != null) { + return timestamp; + } else { + throw new UnsupportedOperationException("No timestamp available."); + } + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException("No windows available."); + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index a2bab2b3060f..88308098bee9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -17,45 +17,174 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.Combine; +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.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +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.GroupCombineFunction; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; +import java.util.Map; /** - * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a - * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements VI, extracts the key and emits accumulated - * values which have the intermediate format VA. + * 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. + * + *

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 implements GroupCombineFunction, KV> { +public class FlinkPartialReduceFunction + extends RichGroupCombineFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; + + protected final WindowingStrategy windowingStrategy; + + protected final SerializedPipelineOptions serializedOptions; - private final Combine.KeyedCombineFn keyedCombineFn; + protected final Map, WindowingStrategy> sideInputs; - public FlinkPartialReduceFunction(Combine.KeyedCombineFn - keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + public FlinkPartialReduceFunction( + CombineFnBase.PerKeyCombineFn combineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + + this.combineFn = combineFn; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; } @Override - public void combine(Iterable> elements, Collector> out) throws Exception { + public void combine( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) 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>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> 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>> iterator = sortedInput.iterator(); - final Iterator> iterator = elements.iterator(); // create accumulator using the first elements key - KV first = iterator.next(); - K key = first.getKey(); - VI value = first.getValue(); - VA accumulator = keyedCombineFn.createAccumulator(key); - accumulator = keyedCombineFn.addInput(key, accumulator, value); - - while(iterator.hasNext()) { - value = iterator.next().getValue(); - accumulator = keyedCombineFn.addInput(key, accumulator, value); + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + 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; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + + // we have to keep track so that we can set the context to the right + // windowed value when windows change in the iterable + currentValue = nextValue; } - out.collect(KV.of(key, accumulator)); + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, accumulator), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java new file mode 100644 index 000000000000..0f1885ca5192 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.flink.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; + +/** + * {@link org.apache.beam.sdk.transforms.DoFn.ProcessContext} for our Flink Wrappers. + */ +class FlinkProcessContext + extends DoFn.ProcessContext { + + private final PipelineOptions pipelineOptions; + private final RuntimeContext runtimeContext; + private Collector> collector; + private final boolean requiresWindowAccess; + + protected WindowedValue windowedValue; + + protected WindowingStrategy windowingStrategy; + + private final Map, WindowingStrategy> sideInputs; + + FlinkProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Collector> collector, + Map, WindowingStrategy> sideInputs) { + doFn.super(); + Preconditions.checkNotNull(pipelineOptions); + Preconditions.checkNotNull(runtimeContext); + Preconditions.checkNotNull(doFn); + Preconditions.checkNotNull(collector); + + this.pipelineOptions = pipelineOptions; + this.runtimeContext = runtimeContext; + this.collector = collector; + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + super.setupDelegateAggregators(); + } + + FlinkProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs) { + doFn.super(); + Preconditions.checkNotNull(pipelineOptions); + Preconditions.checkNotNull(runtimeContext); + Preconditions.checkNotNull(doFn); + + this.pipelineOptions = pipelineOptions; + this.runtimeContext = runtimeContext; + this.collector = null; + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + super.setupDelegateAggregators(); + } + + public FlinkProcessContext forOutput( + Collector> collector) { + this.collector = collector; + + // for now, returns ourselves, to be easy on the GC + return this; + } + + + + public FlinkProcessContext forWindowedValue( + WindowedValue windowedValue) { + this.windowedValue = windowedValue; + + // for now, returns ourselves, to be easy on the GC + return this; + } + + @Override + public InputT element() { + return this.windowedValue.getValue(); + } + + + @Override + public Instant timestamp() { + return windowedValue.getTimestamp(); + } + + @Override + public BoundedWindow window() { + if (!requiresWindowAccess) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } + return Iterables.getOnlyElement(windowedValue.getWindows()); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public WindowingInternals windowingInternals() { + + return new WindowingInternals() { + + @Override + public StateInternals stateInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWindowedValue( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect(WindowedValue.of(value, timestamp, windows, pane)); + outputWithTimestampAndWindow(value, timestamp, windows, pane); + } + + @Override + public TimerInternals timerInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public Collection windows() { + return windowedValue.getWindows(); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public void writePCollectionViewData(TupleTag tag, + Iterable> data, Coder elemCoder) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public ViewT sideInput( + PCollectionView view, + BoundedWindow mainInputWindow) { + + Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + Preconditions.checkNotNull( + sideInputs.get(view), + "Side input for " + view + " not available."); + + // get the side input strategy for mapping the window + WindowingStrategy windowingStrategy = sideInputs.get(view); + + BoundedWindow sideInputWindow = + windowingStrategy.getWindowFn().getSideInputWindow(mainInputWindow); + + Map sideInputs = + runtimeContext.getBroadcastVariableWithInitializer( + view.getTagInternal().getId(), new SideInputInitializer<>(view)); + return sideInputs.get(sideInputWindow); + } + }; + } + + @Override + public PipelineOptions getPipelineOptions() { + return pipelineOptions; + } + + @Override + public ViewT sideInput(PCollectionView view) { + Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + Preconditions.checkNotNull(sideInputs.get(view), "Side input for " + view + " not available."); + Iterator windowIter = windowedValue.getWindows().iterator(); + BoundedWindow window; + if (!windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is not in any windows"); + } else { + window = windowIter.next(); + if (windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is in multiple windows"); + } + } + + // get the side input strategy for mapping the window + WindowingStrategy windowingStrategy = sideInputs.get(view); + + BoundedWindow sideInputWindow = + windowingStrategy.getWindowFn().getSideInputWindow(window); + + Map sideInputs = + runtimeContext.getBroadcastVariableWithInitializer( + view.getTagInternal().getId(), new SideInputInitializer<>(view)); + ViewT result = sideInputs.get(sideInputWindow); + if (result == null) { + result = view.fromIterableInternal(Collections.>emptyList()); + } + return result; + } + + @Override + public void output(OutputT value) { + if (windowedValue != null) { + outputWithTimestamp(value, windowedValue.getTimestamp()); + } else { + outputWithTimestamp(value, null); + } + } + + @Override + public void outputWithTimestamp(OutputT value, Instant timestamp) { + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + value, + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + value, + timestamp, + windowedValue.getWindows(), + windowedValue.getPane())); + } + } + + protected void outputWithTimestampAndWindow( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect( + WindowedValue.of( + value, timestamp, windows, pane)); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException(); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator + createAggregatorInternal(String name, Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = + new SerializableFnAggregatorWrapper<>(combiner); + runtimeContext.addAccumulator(name, wrapper); + return wrapper; + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 43e458fc3720..9cbc6b914765 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -17,43 +17,179 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.Combine; +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.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +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 com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; +import java.util.List; +import java.util.Map; /** - * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a - * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and merges the - * accumulators resulting from the PartialReduce which produced the input VA. + * 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. + * + *

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 implements GroupReduceFunction, KV> { +public class FlinkReduceFunction + extends RichGroupReduceFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; + + protected final WindowingStrategy windowingStrategy; + + protected final Map, WindowingStrategy> sideInputs; + + protected final SerializedPipelineOptions serializedOptions; - private final Combine.KeyedCombineFn keyedCombineFn; + public FlinkReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + this.combineFn = keyedCombineFn; + + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; } @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - Iterator> it = values.iterator(); + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); - KV current = it.next(); - K k = current.getKey(); - VA accumulator = current.getValue(); + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); - while (it.hasNext()) { - current = it.next(); - keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + ArrayList>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> 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>> iterator = sortedInput.iterator(); + + // get the first accumulator + WindowedValue> 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 windowTimestamps = new ArrayList<>(); + windowTimestamps.add(currentValue.getTimestamp()); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.mergeAccumulators( + key, ImmutableList.of(accumulator, nextValue.getValue().getValue()), processContext); + + windowTimestamps.add(nextValue.getTimestamp()); + } else { + // emit the value that we currently have + processContext = processContext.forWindowedValue(currentValue); + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); + + windowTimestamps.clear(); + + currentWindow = nextWindow; + accumulator = nextValue.getValue().getValue(); + windowTimestamps.add(nextValue.getTimestamp()); + } + + // we have to keep track so that we can set the context to the right + // windowed value when windows change in the iterable + currentValue = nextValue; } - out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); + // if at the end of the iteration we have a change in windows + // the ProcessContext will not have been updated + processContext = processContext.forWindowedValue(currentValue); + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); } } 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 new file mode 100644 index 000000000000..451b31b12c5e --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.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.translation.functions; + +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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map} + * from window to side input. + */ +public class SideInputInitializer + implements BroadcastVariableInitializer, Map> { + + PCollectionView view; + + public SideInputInitializer(PCollectionView view) { + this.view = view; + } + + @Override + public Map initializeBroadcastVariable( + Iterable> inputValues) { + + // first partition into windows + Map>> partitionedElements = new HashMap<>(); + for (WindowedValue value: inputValues) { + for (BoundedWindow window: value.getWindows()) { + List> windowedValues = partitionedElements.get(window); + if (windowedValues == null) { + windowedValues = new ArrayList<>(); + partitionedElements.put(window, windowedValues); + } + windowedValues.add(value); + } + } + + Map resultMap = new HashMap<>(); + + for (Map.Entry>> elements: + partitionedElements.entrySet()) { + + @SuppressWarnings("unchecked") + Iterable> elementsIterable = + (List>) (List) elements.getValue(); + + resultMap.put(elements.getKey(), view.fromIterableInternal(elementsIterable)); + } + + return resultMap; + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java deleted file mode 100644 index cc6fd8b70917..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES 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.coders.Coder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - -/** - * A UnionCoder encodes RawUnionValues. - * - * This file copied from {@link org.apache.beam.sdk.transforms.join.UnionCoder} - */ -@SuppressWarnings("serial") -public class UnionCoder extends StandardCoder { - // TODO: Think about how to integrate this with a schema object (i.e. - // a tuple of tuple tags). - /** - * Builds a union coder with the given list of element coders. This list - * corresponds to a mapping of union tag to Coder. Union tags start at 0. - */ - public static UnionCoder of(List> elementCoders) { - return new UnionCoder(elementCoders); - } - - @JsonCreator - public static UnionCoder jsonOf( - @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) - List> elements) { - return UnionCoder.of(elements); - } - - private int getIndexForEncoding(RawUnionValue union) { - if (union == null) { - throw new IllegalArgumentException("cannot encode a null tagged union"); - } - int index = union.getUnionTag(); - if (index < 0 || index >= elementCoders.size()) { - throw new IllegalArgumentException( - "union value index " + index + " not in range [0.." + - (elementCoders.size() - 1) + "]"); - } - return index; - } - - @SuppressWarnings("unchecked") - @Override - public void encode( - RawUnionValue union, - OutputStream outStream, - Context context) - throws IOException { - int index = getIndexForEncoding(union); - // Write out the union tag. - VarInt.encode(index, outStream); - - // Write out the actual value. - Coder coder = (Coder) elementCoders.get(index); - coder.encode( - union.getValue(), - outStream, - context); - } - - @Override - public RawUnionValue decode(InputStream inStream, Context context) - throws IOException { - int index = VarInt.decodeInt(inStream); - Object value = elementCoders.get(index).decode(inStream, context); - return new RawUnionValue(index, value); - } - - @Override - public List> getCoderArguments() { - return null; - } - - @Override - public List> getComponents() { - return elementCoders; - } - - /** - * Since this coder uses elementCoders.get(index) and coders that are known to run in constant - * time, we defer the return value to that coder. - */ - @Override - public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) { - int index = getIndexForEncoding(union); - @SuppressWarnings("unchecked") - Coder coder = (Coder) elementCoders.get(index); - return coder.isRegisterByteSizeObserverCheap(union.getValue(), context); - } - - /** - * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. - */ - @Override - public void registerByteSizeObserver( - RawUnionValue union, ElementByteSizeObserver observer, Context context) - throws Exception { - int index = getIndexForEncoding(union); - // Write out the union tag. - observer.update(VarInt.getLength(index)); - // Write out the actual value. - @SuppressWarnings("unchecked") - Coder coder = (Coder) elementCoders.get(index); - coder.registerByteSizeObserver(union.getValue(), observer, context); - } - - ///////////////////////////////////////////////////////////////////////////// - - private final List> elementCoders; - - private UnionCoder(List> elementCoders) { - this.elementCoders = elementCoders; - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - verifyDeterministic( - "UnionCoder is only deterministic if all element coders are", - elementCoders); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 895ecef1b92e..f03910bf425c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; import com.google.common.base.Preconditions; @@ -71,9 +70,6 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") public TypeSerializer createSerializer(ExecutionConfig config) { - if (coder instanceof VoidCoder) { - return (TypeSerializer) new VoidCoderTypeSerializer(); - } return new CoderTypeSerializer<>(coder); } @@ -84,8 +80,12 @@ public int getTotalFields() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CoderTypeInformation that = (CoderTypeInformation) o; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index c6f3921971a6..097316b242fd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -33,7 +33,7 @@ /** * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for - * Dataflow {@link org.apache.beam.sdk.coders.Coder}s + * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. */ public class CoderTypeSerializer extends TypeSerializer { @@ -128,14 +128,20 @@ public T deserialize(T t, DataInputView dataInputView) throws IOException { } @Override - public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { + 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; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CoderTypeSerializer that = (CoderTypeSerializer) o; return coder.equals(that.coder); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java index 6f0c651406a2..79b127d1062c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java @@ -20,6 +20,8 @@ import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -31,14 +33,13 @@ import java.io.ObjectInputStream; /** - * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for - * {@link org.apache.beam.sdk.coders.KvCoder}. We have a special comparator + * Flink {@link TypeComparator} for {@link KvCoder}. We have a special comparator * for {@link KV} that always compares on the key only. */ -public class KvCoderComperator extends TypeComparator> { +public class KvCoderComperator extends TypeComparator>> { - private KvCoder coder; - private Coder keyCoder; + private final WindowedValue.WindowedValueCoder> coder; + private final Coder keyCoder; // We use these for internal encoding/decoding for creating copies and comparing // serialized forms using a Coder @@ -52,9 +53,10 @@ public class KvCoderComperator extends TypeComparator> { // For deserializing the key private transient DataInputViewWrapper inputWrapper; - public KvCoderComperator(KvCoder coder) { + public KvCoderComperator(WindowedValue.WindowedValueCoder> coder) { this.coder = coder; - this.keyCoder = coder.getKeyCoder(); + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); + this.keyCoder = kvCoder.getKeyCoder(); buffer1 = new InspectableByteArrayOutputStream(); buffer2 = new InspectableByteArrayOutputStream(); @@ -74,8 +76,8 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE } @Override - public int hash(KV record) { - K key = record.getKey(); + public int hash(WindowedValue> record) { + K key = record.getValue().getKey(); if (key != null) { return key.hashCode(); } else { @@ -84,27 +86,27 @@ public int hash(KV record) { } @Override - public void setReference(KV toCompare) { + public void setReference(WindowedValue> toCompare) { referenceBuffer.reset(); try { - keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER); + keyCoder.encode(toCompare.getValue().getKey(), referenceBuffer, Coder.Context.OUTER); } catch (IOException e) { throw new RuntimeException("Could not set reference " + toCompare + ": " + e); } } @Override - public boolean equalToReference(KV candidate) { + public boolean equalToReference(WindowedValue> candidate) { try { buffer2.reset(); - keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER); + keyCoder.encode(candidate.getValue().getKey(), buffer2, Coder.Context.OUTER); byte[] arr = referenceBuffer.getBuffer(); byte[] arrOther = buffer2.getBuffer(); if (referenceBuffer.size() != buffer2.size()) { return false; } int len = buffer2.size(); - for(int i = 0; i < len; i++ ) { + for (int i = 0; i < len; i++) { if (arr[i] != arrOther[i]) { return false; } @@ -116,8 +118,9 @@ public boolean equalToReference(KV candidate) { } @Override - public int compareToReference(TypeComparator> other) { - InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator) other).referenceBuffer; + public int compareToReference(TypeComparator>> other) { + InspectableByteArrayOutputStream otherReferenceBuffer = + ((KvCoderComperator) other).referenceBuffer; byte[] arr = referenceBuffer.getBuffer(); byte[] arrOther = otherReferenceBuffer.getBuffer(); @@ -135,19 +138,19 @@ public int compareToReference(TypeComparator> other) { @Override - public int compare(KV first, KV second) { + public int compare(WindowedValue> first, WindowedValue> second) { try { buffer1.reset(); buffer2.reset(); - keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER); - keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER); + keyCoder.encode(first.getValue().getKey(), buffer1, Coder.Context.OUTER); + keyCoder.encode(second.getValue().getKey(), buffer2, Coder.Context.OUTER); byte[] arr = buffer1.getBuffer(); byte[] arrOther = buffer2.getBuffer(); if (buffer1.size() != buffer2.size()) { return buffer1.size() - buffer2.size(); } int len = buffer1.size(); - for(int i = 0; i < len; i++ ) { + for (int i = 0; i < len; i++) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; } @@ -159,38 +162,19 @@ public int compare(KV first, KV second) { } @Override - public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - + public int compareSerialized( + DataInputView firstSource, + DataInputView secondSource) throws IOException { inputWrapper.setInputView(firstSource); - K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); + WindowedValue> first = coder.decode(inputWrapper, Coder.Context.NESTED); inputWrapper.setInputView(secondSource); - K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); - - try { - buffer1.reset(); - buffer2.reset(); - keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER); - keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER); - byte[] arr = buffer1.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (buffer1.size() != buffer2.size()) { - return buffer1.size() - buffer2.size(); - } - int len = buffer1.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); - } + WindowedValue> second = coder.decode(inputWrapper, Coder.Context.NESTED); + return compare(first, second); } @Override public boolean supportsNormalizedKey() { - return true; + return false; } @Override @@ -209,12 +193,18 @@ public boolean isNormalizedKeyPrefixOnly(int keyBytes) { } @Override - public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + public void putNormalizedKey( + WindowedValue> record, + MemorySegment target, + int offset, + int numBytes) { + buffer1.reset(); try { - keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED); + keyCoder.encode(record.getValue().getKey(), buffer1, Coder.Context.NESTED); } catch (IOException e) { - throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); + throw new RuntimeException( + "Could not serializer " + record + " using coder " + coder + ": " + e); } final byte[] data = buffer1.getBuffer(); final int limit = offset + numBytes; @@ -231,12 +221,16 @@ public void putNormalizedKey(KV record, MemorySegment target, int offset, } @Override - public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { + public void writeWithKeyNormalization( + WindowedValue> record, + DataOutputView target) throws IOException { throw new UnsupportedOperationException(); } @Override - public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { + public WindowedValue> readWithKeyDenormalization( + WindowedValue> reuse, + DataInputView source) throws IOException { throw new UnsupportedOperationException(); } @@ -246,14 +240,14 @@ public boolean invertNormalizedKey() { } @Override - public TypeComparator> duplicate() { + public TypeComparator>> duplicate() { return new KvCoderComperator<>(coder); } @Override public int extractKeys(Object record, Object[] target, int index) { - KV kv = (KV) record; - K k = kv.getKey(); + WindowedValue> kv = (WindowedValue>) record; + K k = kv.getValue().getKey(); target[index] = k; return 1; } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java index 74f3821dfb2f..ba53f640bb81 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import com.google.common.base.Preconditions; @@ -31,27 +32,32 @@ import java.util.List; /** - * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for - * Dataflow {@link org.apache.beam.sdk.coders.KvCoder}. + * Flink {@link TypeInformation} for {@link KvCoder}. This creates special comparator + * for {@link KV} that always compares on the key only. */ -public class KvCoderTypeInformation extends CompositeType> { +public class KvCoderTypeInformation extends CompositeType>> { - private KvCoder coder; + private final WindowedValue.WindowedValueCoder> coder; +// private KvCoder coder; // We don't have the Class, so we have to pass null here. What a shame... - private static Object DUMMY = new Object(); + private static Object dummy = new Object(); @SuppressWarnings("unchecked") - public KvCoderTypeInformation(KvCoder coder) { - super(((Class>) DUMMY.getClass())); + public KvCoderTypeInformation(WindowedValue.WindowedValueCoder> coder) { + super((Class) dummy.getClass()); this.coder = coder; Preconditions.checkNotNull(coder); } @Override @SuppressWarnings("unchecked") - public TypeComparator> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { - return new KvCoderComperator((KvCoder) coder); + public TypeComparator>> createComparator( + int[] logicalKeyFields, + boolean[] orders, + int logicalFieldOffset, + ExecutionConfig config) { + return new KvCoderComperator(coder); } @Override @@ -71,7 +77,7 @@ public int getArity() { @Override @SuppressWarnings("unchecked") - public Class> getTypeClass() { + public Class>> getTypeClass() { return privateGetTypeClass(); } @@ -87,7 +93,7 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer> createSerializer(ExecutionConfig config) { + public TypeSerializer>> createSerializer(ExecutionConfig config) { return new CoderTypeSerializer<>(coder); } @@ -98,8 +104,12 @@ public int getTotalFields() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } KvCoderTypeInformation that = (KvCoderTypeInformation) o; @@ -122,10 +132,11 @@ public String toString() { @Override @SuppressWarnings("unchecked") public TypeInformation getTypeAt(int pos) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); if (pos == 0) { - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getKeyCoder()); } else if (pos == 1) { - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getValueCoder()); } else { throw new RuntimeException("Invalid field position " + pos); } @@ -134,11 +145,12 @@ public TypeInformation getTypeAt(int pos) { @Override @SuppressWarnings("unchecked") public TypeInformation getTypeAt(String fieldExpression) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); switch (fieldExpression) { case "key": - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getKeyCoder()); case "value": - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getValueCoder()); default: throw new UnsupportedOperationException("Only KvCoder has fields."); } @@ -162,17 +174,24 @@ public int getFieldIndex(String fieldName) { } @Override - public void getFlatFields(String fieldExpression, int offset, List result) { - CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); + public void getFlatFields( + String fieldExpression, + int offset, + List result) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); + + CoderTypeInformation keyTypeInfo = + new CoderTypeInformation<>(kvCoder.getKeyCoder()); result.add(new FlatFieldDescriptor(0, keyTypeInfo)); } @Override - protected TypeComparatorBuilder> createTypeComparatorBuilder() { + protected TypeComparatorBuilder>> createTypeComparatorBuilder() { return new KvCoderTypeComparatorBuilder(); } - private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder> { + private class KvCoderTypeComparatorBuilder + implements TypeComparatorBuilder>> { @Override public void initializeTypeComparatorBuilder(int size) {} @@ -181,7 +200,7 @@ public void initializeTypeComparatorBuilder(int size) {} public void addComparatorField(int fieldId, TypeComparator comparator) {} @Override - public TypeComparator> createTypeComparator(ExecutionConfig config) { + public TypeComparator>> createTypeComparator(ExecutionConfig config) { return new KvCoderComperator<>(coder); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java deleted file mode 100644 index e5567d3ea3b2..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES 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.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; - -import com.google.common.collect.Lists; - -import org.apache.flink.api.common.accumulators.Accumulator; - -import java.io.Serializable; - -/** - * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn} - * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using - * the combine function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} - * operation. - */ -public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { - - private AA aa; - private Combine.CombineFn combiner; - - public CombineFnAggregatorWrapper() { - } - - public CombineFnAggregatorWrapper(Combine.CombineFn combiner) { - this.combiner = combiner; - this.aa = combiner.createAccumulator(); - } - - @Override - public void add(AI value) { - combiner.addInput(aa, value); - } - - @Override - public Serializable getLocalValue() { - return (Serializable) combiner.extractOutput(aa); - } - - @Override - public void resetLocal() { - aa = combiner.createAccumulator(); - } - - @Override - @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); - } - - @Override - public Accumulator clone() { - // copy it by merging - AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa)); - CombineFnAggregatorWrapper result = new - CombineFnAggregatorWrapper<>(combiner); - result.aa = aaCopy; - return result; - } - - @Override - public void addValue(AI value) { - add(value); - } - - @Override - public String getName() { - return "CombineFn: " + combiner.toString(); - } - - @Override - public Combine.CombineFn getCombineFn() { - return combiner; - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index eb32fa2fd74a..82d3fb8ffae3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -33,20 +33,21 @@ * the function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} * operation. */ -public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { +public class SerializableFnAggregatorWrapper + implements Aggregator, Accumulator { - private AO aa; - private Combine.CombineFn combiner; + private OutputT aa; + private Combine.CombineFn combiner; - public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { + public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { this.combiner = combiner; resetLocal(); } - + @Override @SuppressWarnings("unchecked") - public void add(AI value) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, value)); + public void add(InputT value) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, value)); } @Override @@ -56,17 +57,17 @@ public Serializable getLocalValue() { @Override public void resetLocal() { - this.aa = combiner.apply(ImmutableList.of()); + this.aa = combiner.apply(ImmutableList.of()); } @Override @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); + public void merge(Accumulator other) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, (InputT) other.getLocalValue())); } @Override - public void addValue(AI value) { + public void addValue(InputT value) { add(value); } @@ -76,15 +77,15 @@ public String getName() { } @Override - public Combine.CombineFn getCombineFn() { + public Combine.CombineFn getCombineFn() { return combiner; } @Override - public Accumulator clone() { + public Accumulator clone() { // copy it by merging - AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); - SerializableFnAggregatorWrapper result = new + OutputT resultCopy = combiner.apply(Lists.newArrayList((InputT) aa)); + SerializableFnAggregatorWrapper result = new SerializableFnAggregatorWrapper<>(combiner); result.aa = resultCopy; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java index 2766a873e35f..28a4b9ee59df 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java @@ -22,6 +22,7 @@ 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.util.WindowedValue; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; @@ -31,10 +32,11 @@ import java.lang.reflect.Field; /** - * Wrapper class to use generic Write.Bound transforms as sinks. + * Wrapper for executing a {@link Sink} on Flink as an {@link OutputFormat}. + * * @param The type of the incoming records. */ -public class SinkOutputFormat implements OutputFormat { +public class SinkOutputFormat implements OutputFormat> { private final Sink sink; @@ -88,9 +90,9 @@ public void open(int taskNumber, int numTasks) throws IOException { } @Override - public void writeRecord(T record) throws IOException { + public void writeRecord(WindowedValue record) throws IOException { try { - writer.write(record); + writer.write(record.getValue()); } catch (Exception e) { throw new IOException("Couldn't write record.", e); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index debd1a14d525..1d06b1ac2fc9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -21,12 +21,16 @@ 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; @@ -35,10 +39,10 @@ /** - * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a - * Dataflow {@link org.apache.beam.sdk.io.Source}. + * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. */ -public class SourceInputFormat implements InputFormat> { +public class SourceInputFormat + implements InputFormat, SourceInputSplit> { private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); private final BoundedSource initialSource; @@ -122,12 +126,16 @@ public boolean reachedEnd() throws IOException { } @Override - public T nextRecord(T t) throws IOException { + public WindowedValue nextRecord(WindowedValue 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 current; + return WindowedValue.of( + current, + timestamp, + GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } return null; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java index 113fee0881de..d7b60365b349 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java @@ -30,6 +30,9 @@ import org.apache.flink.test.util.JavaProgramTestBase; +import java.io.File; +import java.net.URI; + public class AvroITCase extends JavaProgramTestBase { @@ -51,6 +54,15 @@ protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); tmpPath = getTempDirPath("tmp"); + // need to create the dirs, otherwise Beam sinks don't + // work for these tests + if (!new File(new URI(tmpPath)).mkdirs()) { + throw new RuntimeException("Could not create temp output dir."); + } + + if (!new File(new URI(resultPath)).mkdirs()) { + throw new RuntimeException("Could not create output dir."); + } } @Override @@ -63,27 +75,25 @@ protected void testProgram() throws Exception { runProgram(tmpPath, resultPath); } - private static void runProgram(String tmpPath, String resultPath) { + private static void runProgram(String tmpPath, String resultPath) throws Exception { Pipeline p = FlinkTestPipeline.createForBatch(); p - .apply(Create.of( - new User("Joe", 3, "red"), - new User("Mary", 4, "blue"), - new User("Mark", 1, "green"), - new User("Julia", 5, "purple")) - .withCoder(AvroCoder.of(User.class))) - - .apply(AvroIO.Write.to(tmpPath) - .withSchema(User.class)); + .apply(Create.of( + new User("Joe", 3, "red"), + new User("Mary", 4, "blue"), + new User("Mark", 1, "green"), + new User("Julia", 5, "purple")) + .withCoder(AvroCoder.of(User.class))) + .apply(AvroIO.Write.to(new URI(tmpPath).getPath() + "/part") + .withSchema(User.class)); p.run(); p = FlinkTestPipeline.createForBatch(); p - .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation()) - + .apply(AvroIO.Read.from(tmpPath + "/*").withSchema(User.class)) .apply(ParDo.of(new DoFn() { @Override public void processElement(ProcessContext c) throws Exception { @@ -92,8 +102,8 @@ public void processElement(ProcessContext c) throws Exception { c.output(result); } })) + .apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); - .apply(TextIO.Write.to(resultPath)); p.run(); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java index 4d66fa421c5e..53fac339c95f 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java @@ -26,7 +26,9 @@ import org.apache.flink.test.util.JavaProgramTestBase; +import java.io.File; import java.io.Serializable; +import java.net.URI; public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable { @@ -40,6 +42,11 @@ public MaybeEmptyTestITCase() { @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); + // need to create the dirs, 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 @@ -59,7 +66,7 @@ protected void testProgram() throws Exception { public void processElement(DoFn.ProcessContext c) { c.output(expected); } - })).apply(TextIO.Write.to(resultPath)); + })).apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); p.run(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 43f1adf0c0c8..587c041d0654 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -35,7 +35,7 @@ /** * A UnionCoder encodes RawUnionValues. */ -class UnionCoder extends StandardCoder { +public class UnionCoder extends StandardCoder { // TODO: Think about how to integrate this with a schema object (i.e. // a tuple of tuple tags). /** From 072343d6e9bc0105177cea908bd34740aa6c3e21 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 13 May 2016 14:41:20 +0200 Subject: [PATCH 28/29] Remove superfluous Flink Tests, Fix those that stay in All of the stuff in the removed ITCases is covered (in more detail) by the RunnableOnService tests. --- .../apache/beam/runners/flink/AvroITCase.java | 139 --------------- .../beam/runners/flink/FlattenizeITCase.java | 76 --------- .../runners/flink/JoinExamplesITCase.java | 102 ----------- .../runners/flink/MaybeEmptyTestITCase.java | 73 -------- .../runners/flink/ParDoMultiOutputITCase.java | 102 ----------- .../beam/runners/flink/ReadSourceITCase.java | 14 +- .../flink/RemoveDuplicatesEmptyITCase.java | 72 -------- .../runners/flink/RemoveDuplicatesITCase.java | 73 -------- .../beam/runners/flink/SideInputITCase.java | 70 -------- .../beam/runners/flink/TfIdfITCase.java | 80 --------- .../beam/runners/flink/WordCountITCase.java | 77 --------- .../runners/flink/WordCountJoin2ITCase.java | 140 --------------- .../runners/flink/WordCountJoin3ITCase.java | 158 ----------------- .../streaming/GroupAlsoByWindowTest.java | 3 +- .../beam/runners/flink/util/JoinExamples.java | 161 ------------------ 15 files changed, 14 insertions(+), 1326 deletions(-) delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java deleted file mode 100644 index d7b60365b349..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.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.coders.AvroCoder; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.File; -import java.net.URI; - - -public class AvroITCase extends JavaProgramTestBase { - - protected String resultPath; - protected String tmpPath; - - public AvroITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "Joe red 3", - "Mary blue 4", - "Mark green 1", - "Julia purple 5" - }; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - tmpPath = getTempDirPath("tmp"); - - // need to create the dirs, otherwise Beam sinks don't - // work for these tests - if (!new File(new URI(tmpPath)).mkdirs()) { - throw new RuntimeException("Could not create temp output dir."); - } - - 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(tmpPath, resultPath); - } - - private static void runProgram(String tmpPath, String resultPath) throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - p - .apply(Create.of( - new User("Joe", 3, "red"), - new User("Mary", 4, "blue"), - new User("Mark", 1, "green"), - new User("Julia", 5, "purple")) - .withCoder(AvroCoder.of(User.class))) - .apply(AvroIO.Write.to(new URI(tmpPath).getPath() + "/part") - .withSchema(User.class)); - - p.run(); - - p = FlinkTestPipeline.createForBatch(); - - p - .apply(AvroIO.Read.from(tmpPath + "/*").withSchema(User.class)) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - User u = c.element(); - String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber(); - c.output(result); - } - })) - .apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); - - - p.run(); - } - - private static class User { - - private String name; - private int favoriteNumber; - private String favoriteColor; - - public User() {} - - public User(String name, int favoriteNumber, String favoriteColor) { - this.name = name; - this.favoriteNumber = favoriteNumber; - this.favoriteColor = favoriteColor; - } - - public String getName() { - return name; - } - - public String getFavoriteColor() { - return favoriteColor; - } - - public int getFavoriteNumber() { - return favoriteNumber; - } - } - -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java deleted file mode 100644 index ac0a3d7d4d67..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -public class FlattenizeITCase extends JavaProgramTestBase { - - private String resultPath; - private String resultPath2; - - private static final String[] words = {"hello", "this", "is", "a", "DataSet!"}; - private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"}; - private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - resultPath2 = getTempDirPath("result2"); - } - - @Override - protected void postSubmit() throws Exception { - String join = Joiner.on('\n').join(words); - String join2 = Joiner.on('\n').join(words2); - String join3 = Joiner.on('\n').join(words3); - compareResultsByLinesInMemory(join + "\n" + join2, resultPath); - compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2); - } - - - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection p1 = p.apply(Create.of(words)); - PCollection p2 = p.apply(Create.of(words2)); - - PCollectionList list = PCollectionList.of(p1).and(p2); - - list.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath)); - - PCollection p3 = p.apply(Create.of(words3)); - - PCollectionList list2 = list.and(p3); - - list2.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath2)); - - p.run(); - } - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java deleted file mode 100644 index 47685b6be6f3..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.runners.flink.util.JoinExamples; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -/** - * Unfortunately we need to copy the code from the Dataflow SDK because it is not public there. - */ -public class JoinExamplesITCase extends JavaProgramTestBase { - - protected String resultPath; - - public JoinExamplesITCase(){ - } - - private static final TableRow row1 = new TableRow() - .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") - .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com"); - private static final TableRow row2 = new TableRow() - .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") - .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com"); - private static final TableRow row3 = new TableRow() - .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213") - .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com"); - static final TableRow[] EVENTS = new TableRow[] { - row1, row2, row3 - }; - static final List EVENT_ARRAY = Arrays.asList(EVENTS); - - private static final TableRow cc1 = new TableRow() - .set("FIPSCC", "VM").set("HumanName", "Vietnam"); - private static final TableRow cc2 = new TableRow() - .set("FIPSCC", "BE").set("HumanName", "Belgium"); - static final TableRow[] CCS = new TableRow[] { - cc1, cc2 - }; - static final List CC_ARRAY = Arrays.asList(CCS); - - static final String[] JOINED_EVENTS = new String[] { - "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, " - + "url: http://www.chicagotribune.com", - "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, " - + "url: http://cnn.com", - "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, " - + "url: http://cnn.com" - }; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input1 = p.apply(Create.of(EVENT_ARRAY)); - PCollection input2 = p.apply(Create.of(CC_ARRAY)); - - PCollection output = JoinExamples.joinEvents(input1, input2); - - output.apply(TextIO.Write.to(resultPath)); - - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java deleted file mode 100644 index 53fac339c95f..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.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; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.File; -import java.io.Serializable; -import java.net.URI; - -public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable { - - protected String resultPath; - - protected final String expected = "test"; - - public MaybeEmptyTestITCase() { - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - // need to create the dirs, 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(expected, resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) - .apply(ParDo.of( - new DoFn() { - @Override - public void processElement(DoFn.ProcessContext c) { - c.output(expected); - } - })).apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); - p.run(); - } - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java deleted file mode 100644 index a2ef4e29f403..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.Serializable; - -public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable { - - private String resultPath; - - private static String[] expectedWords = {"MAAA", "MAAFOOO"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath); - } - - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); - - // Select words whose length is below a cut off, - // plus the lengths of words that are above the cut off. - // Also select words starting with "MARKER". - final int wordLengthCutOff = 3; - // Create tags to use for the main and side outputs. - final TupleTag wordsBelowCutOffTag = new TupleTag(){}; - final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; - final TupleTag markedWordsTag = new TupleTag(){}; - - PCollectionTuple results = - words.apply(ParDo - .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) - .and(markedWordsTag)) - .of(new DoFn() { - final TupleTag specialWordsTag = new TupleTag() { - }; - - public void processElement(ProcessContext c) { - String word = c.element(); - if (word.length() <= wordLengthCutOff) { - c.output(word); - } else { - c.sideOutput(wordLengthsAboveCutOffTag, word.length()); - } - if (word.startsWith("MAA")) { - c.sideOutput(markedWordsTag, word); - } - - if (word.startsWith("SPECIAL")) { - c.sideOutput(specialWordsTag, word); - } - } - })); - - // Extract the PCollection results, by tag. - PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); - PCollection wordLengthsAboveCutOff = results.get - (wordLengthsAboveCutOffTag); - PCollection markedWords = results.get(markedWordsTag); - - markedWords.apply(TextIO.Write.to(resultPath)); - - p.run(); - } -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java index 66c959eea90c..bb79b270945c 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java @@ -28,6 +28,9 @@ import org.apache.flink.test.util.JavaProgramTestBase; +import java.io.File; +import java.net.URI; + /** * Reads from a bounded source in batch execution. */ @@ -44,6 +47,13 @@ public ReadSourceITCase(){ @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 @@ -56,7 +66,7 @@ protected void testProgram() throws Exception { runProgram(resultPath); } - private static void runProgram(String resultPath) { + private static void runProgram(String resultPath) throws Exception { Pipeline p = FlinkTestPipeline.createForBatch(); @@ -69,7 +79,7 @@ public void processElement(ProcessContext c) throws Exception { } })); - result.apply(TextIO.Write.to(resultPath)); + result.apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); p.run(); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java deleted file mode 100644 index 471d3262a36c..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.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.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Collections; -import java.util.List; - - -public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase { - - protected String resultPath; - - public RemoveDuplicatesEmptyITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] {}; - - @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 { - - List strings = Collections.emptyList(); - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); - - PCollection output = - input.apply(RemoveDuplicates.create()); - - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java deleted file mode 100644 index 0544f20eb310..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.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; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -public class RemoveDuplicatesITCase extends JavaProgramTestBase { - - protected String resultPath; - - public RemoveDuplicatesITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "k1", "k5", "k2", "k3"}; - - @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 { - - List strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3"); - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); - - PCollection output = - input.apply(RemoveDuplicates.create()); - - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java deleted file mode 100644 index 2c7c65e8af3d..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.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.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.PCollectionView; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.Serializable; - -public class SideInputITCase extends JavaProgramTestBase implements Serializable { - - private static final String expected = "Hello!"; - - protected String resultPath; - - @Override - protected void testProgram() throws Exception { - - - Pipeline p = FlinkTestPipeline.createForBatch(); - - - final PCollectionView sidesInput = p - .apply(Create.of(expected)) - .apply(View.asSingleton()); - - p.apply(Create.of("bli")) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - String s = c.sideInput(sidesInput); - c.output(s); - } - }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath)); - - p.run(); - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expected, resultPath); - } -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java deleted file mode 100644 index 547f3c3a4660..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.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; - -import org.apache.beam.examples.complete.TfIdf; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringDelegateCoder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Keys; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.net.URI; - - -public class TfIdfITCase extends JavaProgramTestBase { - - protected String resultPath; - - public TfIdfITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "a", "m", "n", "b", "c", "d"}; - - @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 pipeline = FlinkTestPipeline.createForBatch(); - - pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - - PCollection>> wordToUriAndTfIdf = pipeline - .apply(Create.of( - KV.of(new URI("x"), "a b c d"), - KV.of(new URI("y"), "a b c"), - KV.of(new URI("z"), "a m n"))) - .apply(new TfIdf.ComputeTfIdf()); - - PCollection words = wordToUriAndTfIdf - .apply(Keys.create()) - .apply(RemoveDuplicates.create()); - - words.apply(TextIO.Write.to(resultPath)); - - pipeline.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java deleted file mode 100644 index 3254e7885db8..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.examples.WordCount; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -public class WordCountITCase extends JavaProgramTestBase { - - protected String resultPath; - - public WordCountITCase(){ - } - - static final String[] WORDS_ARRAY = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final List WORDS = Arrays.asList(WORDS_ARRAY); - - static final String[] COUNTS_ARRAY = new String[] { - "hi: 5", "there: 1", "sue: 2", "bob: 2"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); - - input - .apply(new WordCount.CountWords()) - .apply(MapElements.via(new WordCount.FormatAsTextFn())) - .apply(TextIO.Write.to(resultPath)); - - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java deleted file mode 100644 index 6570e7df5508..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - - -public class WordCountJoin2ITCase extends JavaProgramTestBase { - - static final String[] WORDS_1 = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final String[] WORDS_2 = new String[] { - "hi tim", "beauty", "hooray sue bob", - "hi there", "", "please say hi"}; - - static final String[] RESULTS = new String[] { - "beauty -> Tag1: Tag2: 1", - "bob -> Tag1: 2 Tag2: 1", - "hi -> Tag1: 5 Tag2: 3", - "hooray -> Tag1: Tag2: 1", - "please -> Tag1: Tag2: 1", - "say -> Tag1: Tag2: 1", - "sue -> Tag1: 2 Tag2: 1", - "there -> Tag1: 1 Tag2: 1", - "tim -> Tag1: Tag2: 1" - }; - - static final TupleTag tag1 = new TupleTag<>("Tag1"); - static final TupleTag tag2 = new TupleTag<>("Tag2"); - - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - /* Create two PCollections and join them */ - PCollection> occurences1 = p.apply(Create.of(WORDS_1)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences2 = p.apply(Create.of(WORDS_2)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - /* CoGroup the two collections */ - PCollection> mergedOccurences = KeyedPCollectionTuple - .of(tag1, occurences1) - .and(tag2, occurences2) - .apply(CoGroupByKey.create()); - - /* Format output */ - mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test").to(resultPath)); - - p.run(); - } - - - static class ExtractWordsFn extends DoFn { - - @Override - public void startBundle(Context c) { - } - - @Override - public void processElement(ProcessContext c) { - // 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); - } - } - } - } - - static class FormatCountsFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - CoGbkResult value = c.element().getValue(); - String key = c.element().getKey(); - String countTag1 = tag1.getId() + ": "; - String countTag2 = tag2.getId() + ": "; - for (Long count : value.getAll(tag1)) { - countTag1 += count + " "; - } - for (Long count : value.getAll(tag2)) { - countTag2 += count; - } - c.output(key + " -> " + countTag1 + countTag2); - } - } - - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java deleted file mode 100644 index 60dc74af90b6..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES 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.io.TextIO; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - - -public class WordCountJoin3ITCase extends JavaProgramTestBase { - - static final String[] WORDS_1 = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final String[] WORDS_2 = new String[] { - "hi tim", "beauty", "hooray sue bob", - "hi there", "", "please say hi"}; - - static final String[] WORDS_3 = new String[] { - "hi stephan", "beauty", "hooray big fabian", - "hi yo", "", "please say hi"}; - - static final String[] RESULTS = new String[] { - "beauty -> Tag1: Tag2: 1 Tag3: 1", - "bob -> Tag1: 2 Tag2: 1 Tag3: ", - "hi -> Tag1: 5 Tag2: 3 Tag3: 3", - "hooray -> Tag1: Tag2: 1 Tag3: 1", - "please -> Tag1: Tag2: 1 Tag3: 1", - "say -> Tag1: Tag2: 1 Tag3: 1", - "sue -> Tag1: 2 Tag2: 1 Tag3: ", - "there -> Tag1: 1 Tag2: 1 Tag3: ", - "tim -> Tag1: Tag2: 1 Tag3: ", - "stephan -> Tag1: Tag2: Tag3: 1", - "yo -> Tag1: Tag2: Tag3: 1", - "fabian -> Tag1: Tag2: Tag3: 1", - "big -> Tag1: Tag2: Tag3: 1" - }; - - static final TupleTag tag1 = new TupleTag<>("Tag1"); - static final TupleTag tag2 = new TupleTag<>("Tag2"); - static final TupleTag tag3 = new TupleTag<>("Tag3"); - - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - /* Create two PCollections and join them */ - PCollection> occurences1 = p.apply(Create.of(WORDS_1)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences2 = p.apply(Create.of(WORDS_2)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences3 = p.apply(Create.of(WORDS_3)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - /* CoGroup the two collections */ - PCollection> mergedOccurences = KeyedPCollectionTuple - .of(tag1, occurences1) - .and(tag2, occurences2) - .and(tag3, occurences3) - .apply(CoGroupByKey.create()); - - /* Format output */ - mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test").to(resultPath)); - - p.run(); - } - - - static class ExtractWordsFn extends DoFn { - - @Override - public void startBundle(Context c) { - } - - @Override - public void processElement(ProcessContext c) { - // 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); - } - } - } - } - - static class FormatCountsFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - CoGbkResult value = c.element().getValue(); - String key = c.element().getKey(); - String countTag1 = tag1.getId() + ": "; - String countTag2 = tag2.getId() + ": "; - String countTag3 = tag3.getId() + ": "; - for (Long count : value.getAll(tag1)) { - countTag1 += count + " "; - } - for (Long count : value.getAll(tag2)) { - countTag2 += count + " "; - } - for (Long count : value.getAll(tag3)) { - countTag3 += count; - } - c.output(key + " -> " + countTag1 + countTag2 + countTag3); - } - } - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java index f3ceba76077a..a488768467ea 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java @@ -43,6 +43,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestHarnessUtil; import org.joda.time.Duration; import org.joda.time.Instant; @@ -52,7 +53,7 @@ import java.util.Comparator; import java.util.concurrent.ConcurrentLinkedQueue; -public class GroupAlsoByWindowTest { +public class GroupAlsoByWindowTest extends StreamingMultipleProgramsTestBase { private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java deleted file mode 100644 index e6b7f64f69a1..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.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.util; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.BigQueryIO; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.services.bigquery.model.TableRow; - -/** - * Copied from {@link org.apache.beam.examples.JoinExamples} because the code - * is private there. - */ -public class JoinExamples { - - // A 1000-row sample of the GDELT data here: gdelt-bq:full.events. - private static final String GDELT_EVENTS_TABLE = - "clouddataflow-readonly:samples.gdelt_sample"; - // A table that maps country codes to country names. - private static final String COUNTRY_CODES = - "gdelt-bq:full.crosswalk_geocountrycodetohuman"; - - /** - * Join two collections, using country code as the key. - */ - public static PCollection joinEvents(PCollection eventsTable, - PCollection countryCodes) throws Exception { - - final TupleTag eventInfoTag = new TupleTag<>(); - final TupleTag countryInfoTag = new TupleTag<>(); - - // transform both input collections to tuple collections, where the keys are country - // codes in both cases. - PCollection> eventInfo = eventsTable.apply( - ParDo.of(new ExtractEventDataFn())); - PCollection> countryInfo = countryCodes.apply( - ParDo.of(new ExtractCountryInfoFn())); - - // country code 'key' -> CGBKR (, ) - PCollection> kvpCollection = KeyedPCollectionTuple - .of(eventInfoTag, eventInfo) - .and(countryInfoTag, countryInfo) - .apply(CoGroupByKey.create()); - - // Process the CoGbkResult elements generated by the CoGroupByKey transform. - // country code 'key' -> string of , - PCollection> finalResultCollection = - kvpCollection.apply(ParDo.of(new DoFn, KV>() { - @Override - public void processElement(ProcessContext c) { - KV e = c.element(); - CoGbkResult val = e.getValue(); - String countryCode = e.getKey(); - String countryName; - countryName = e.getValue().getOnly(countryInfoTag, "Kostas"); - for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { - // Generate a string that combines information from both collection values - c.output(KV.of(countryCode, "Country name: " + countryName - + ", Event info: " + eventInfo)); - } - } - })); - - // write to GCS - return finalResultCollection - .apply(ParDo.of(new DoFn, String>() { - @Override - public void processElement(ProcessContext c) { - String outputstring = "Country code: " + c.element().getKey() - + ", " + c.element().getValue(); - c.output(outputstring); - } - })); - } - - /** - * Examines each row (event) in the input table. Output a KV with the key the country - * code of the event, and the value a string encoding event information. - */ - static class ExtractEventDataFn extends DoFn> { - @Override - public void processElement(ProcessContext c) { - TableRow row = c.element(); - String countryCode = (String) row.get("ActionGeo_CountryCode"); - String sqlDate = (String) row.get("SQLDATE"); - String actor1Name = (String) row.get("Actor1Name"); - String sourceUrl = (String) row.get("SOURCEURL"); - String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl; - c.output(KV.of(countryCode, eventInfo)); - } - } - - - /** - * Examines each row (country info) in the input table. Output a KV with the key the country - * code, and the value the country name. - */ - static class ExtractCountryInfoFn extends DoFn> { - @Override - public void processElement(ProcessContext c) { - TableRow row = c.element(); - String countryCode = (String) row.get("FIPSCC"); - String countryName = (String) row.get("HumanName"); - c.output(KV.of(countryCode, countryName)); - } - } - - - /** - * Options supported by {@link JoinExamples}. - *

- * Inherits standard configuration options. - */ - private interface Options extends PipelineOptions { - @Description("Path of the file to write to") - @Validation.Required - String getOutput(); - void setOutput(String value); - } - - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - Pipeline p = Pipeline.create(options); - // the following two 'applys' create multiple inputs to our pipeline, one for each - // of our two input sources. - PCollection eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE)); - PCollection countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES)); - PCollection formattedResults = joinEvents(eventsTable, countryCodes); - formattedResults.apply(TextIO.Write.to(options.getOutput())); - p.run(); - } - -} From 4292f04ebe30fada9158ec736a58a2ea56feb8ef Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Sat, 14 May 2016 11:48:47 +0200 Subject: [PATCH 29/29] Fix last last outstanding test --- .../FlinkBatchTransformTranslators.java | 117 +++++---- .../functions/FlinkCreateFunction.java | 63 ----- .../FlinkMergingNonShuffleReduceFunction.java | 238 ++++++++++++++++++ .../FlinkMergingPartialReduceFunction.java | 8 +- .../functions/FlinkMergingReduceFunction.java | 12 +- .../functions/FlinkPartialReduceFunction.java | 6 +- .../types/CoderTypeInformation.java | 9 +- .../types/VoidCoderTypeSerializer.java | 112 --------- .../streaming/FlinkGroupByKeyWrapper.java | 10 +- .../io/FlinkStreamingCreateFunction.java | 9 +- 10 files changed, 328 insertions(+), 256 deletions(-) delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index fda4d29556f3..86aee84ab1cb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -19,6 +19,7 @@ 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.FlinkMultiOutputDoFnFunction; @@ -353,6 +354,7 @@ public void translateNode( FlinkReduceFunction, List, ?> reduceFunction; if (windowingStrategy.getWindowFn().isNonMerging()) { + @SuppressWarnings("unchecked") WindowingStrategy boundedStrategy = (WindowingStrategy) windowingStrategy; @@ -373,6 +375,8 @@ public void translateNode( throw new UnsupportedOperationException( "Merging WindowFn with windows other than IntervalWindow are not supported."); } + + @SuppressWarnings("unchecked") WindowingStrategy intervalStrategy = (WindowingStrategy) windowingStrategy; @@ -519,74 +523,91 @@ public void translateNode( sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); } - FlinkPartialReduceFunction partialReduceFunction; - FlinkReduceFunction reduceFunction; - if (windowingStrategy.getWindowFn().isNonMerging()) { WindowingStrategy boundedStrategy = (WindowingStrategy) windowingStrategy; - partialReduceFunction = new FlinkPartialReduceFunction<>( - combineFn, - boundedStrategy, - sideInputStrategies, - context.getPipelineOptions()); - - reduceFunction = new FlinkReduceFunction<>( - combineFn, - boundedStrategy, - sideInputStrategies, - context.getPipelineOptions()); + FlinkPartialReduceFunction partialReduceFunction = + new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + FlinkReduceFunction reduceFunction = + new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); + + transformSideInputs(transform.getSideInputs(), groupCombine, context); + + TypeInformation>> reduceTypeInfo = + context.getTypeInfo(context.getOutput(transform)); + + Grouping>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format OutputT + GroupReduceOperator< + WindowedValue>, WindowedValue>> 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 intervalStrategy = (WindowingStrategy) windowingStrategy; - partialReduceFunction = new FlinkMergingPartialReduceFunction<>( - combineFn, - intervalStrategy, - sideInputStrategies, - context.getPipelineOptions()); + FlinkMergingNonShuffleReduceFunction reduceFunction = + new FlinkMergingNonShuffleReduceFunction<>( + combineFn, + intervalStrategy, + sideInputStrategies, + context.getPipelineOptions()); - reduceFunction = new FlinkMergingReduceFunction<>( - combineFn, - intervalStrategy, - sideInputStrategies, - context.getPipelineOptions()); - } + TypeInformation>> reduceTypeInfo = + context.getTypeInfo(context.getOutput(transform)); - // Partially GroupReduce the values into the intermediate format AccumT (combine) - GroupCombineOperator< - WindowedValue>, - WindowedValue>> groupCombine = - new GroupCombineOperator<>( - inputGrouping, - partialReduceTypeInfo, - partialReduceFunction, - "GroupCombine: " + transform.getName()); - - transformSideInputs(transform.getSideInputs(), groupCombine, context); + Grouping>> grouping = + new UnsortedGrouping<>( + inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); - TypeInformation>> reduceTypeInfo = - context.getTypeInfo(context.getOutput(transform)); + // Fully reduce the values and create output format OutputT + GroupReduceOperator< + WindowedValue>, WindowedValue>> outputDataSet = + new GroupReduceOperator<>( + grouping, reduceTypeInfo, reduceFunction, transform.getName()); - Grouping>> intermediateGrouping = - new UnsortedGrouping<>( - groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + transformSideInputs(transform.getSideInputs(), outputDataSet, context); - // Fully reduce the values and create output format OutputT - GroupReduceOperator< - WindowedValue>, WindowedValue>> outputDataSet = - new GroupReduceOperator<>( - intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } - transformSideInputs(transform.getSideInputs(), outputDataSet, context); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java deleted file mode 100644 index e5ac7482cfcb..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.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 org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; -import org.apache.beam.sdk.coders.Coder; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.util.Collector; - -import java.io.ByteArrayInputStream; -import java.util.List; - -/** - * This is a hack for transforming a {@link org.apache.beam.sdk.transforms.Create} - * operation. Flink does not allow {@code null} in it's equivalent operation: - * {@link org.apache.flink.api.java.ExecutionEnvironment#fromElements(Object[])}. Therefore - * we use a DataSource with one dummy element and output the elements of the Create operation - * inside this FlatMap. - */ -public class FlinkCreateFunction implements FlatMapFunction { - - private final List elements; - private final Coder coder; - - public FlinkCreateFunction(List elements, Coder coder) { - this.elements = elements; - this.coder = coder; - } - - @Override - @SuppressWarnings("unchecked") - public void flatMap(IN value, Collector out) throws Exception { - - for (byte[] element : elements) { - ByteArrayInputStream bai = new ByteArrayInputStream(element); - OUT outValue = coder.decode(bai, Coder.Context.OUTER); - if (outValue == null) { - // TODO Flink doesn't allow null values in records - out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE); - } else { - out.collect(outValue); - } - } - - out.close(); - } -} 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 new file mode 100644 index 000000000000..9074d72e0e15 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +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.DoFn; +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.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +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 com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * 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}. + * + *

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>> { + + private final CombineFnBase.PerKeyCombineFn combineFn; + + private final DoFn, KV> doFn; + + private final WindowingStrategy windowingStrategy; + + private final Map, WindowingStrategy> sideInputs; + + private final SerializedPipelineOptions serializedOptions; + + public FlinkMergingNonShuffleReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + + this.combineFn = keyedCombineFn; + + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; + } + + @Override + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) 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>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> 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>> iterator = sortedInput.iterator(); + + // create accumulator using the first elements key + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (currentWindow.equals(nextWindow)) { + // continue accumulating and merge windows + + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + 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, processContext)), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + + currentWindow = nextWindow; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + } + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + 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>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> 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> 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> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } + +} 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 index 47f920770d45..c12e4204a3f0 100644 --- 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 @@ -85,11 +85,9 @@ public void combine( List>> sortedInput = Lists.newArrayList(); for (WindowedValue> inputValue: elements) { for (WindowedValue> exploded: inputValue.explodeWindows()) { - System.out.println("ADDING EXLODE: " + exploded); sortedInput.add(exploded); } } - System.out.println("SORTED SIZE: " + sortedInput.size()); Collections.sort(sortedInput, new Comparator>>() { @Override public int compare( @@ -148,15 +146,11 @@ public int compare( currentWindow = nextWindow; InputT value = nextValue.getValue().getValue(); - processContext = processContext.forWindowedValue(currentValue); + processContext = processContext.forWindowedValue(nextValue); accumulator = combineFnRunner.createAccumulator(key, processContext); accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } // emit the final accumulator 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 index 3bd08075401f..07d1c9741533 100644 --- 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 @@ -137,8 +137,6 @@ public int compare( windowTimestamps.add(nextValue.getTimestamp()); } else { - // emit the value that we currently have - processContext = processContext.forWindowedValue(currentValue); out.collect( WindowedValue.of( KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), @@ -148,20 +146,14 @@ public int compare( windowTimestamps.clear(); + processContext = processContext.forWindowedValue(nextValue); + currentWindow = nextWindow; accumulator = nextValue.getValue().getValue(); windowTimestamps.add(nextValue.getTimestamp()); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } - // if at the end of the iteration we have a change in windows - // the ProcessContext will not have been updated - processContext = processContext.forWindowedValue(currentValue); - // emit the final accumulator out.collect( WindowedValue.of( diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 88308098bee9..c29e1df2ceb0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -168,15 +168,11 @@ public int compare( currentWindow = nextWindow; InputT value = nextValue.getValue().getValue(); - processContext = processContext.forWindowedValue(currentValue); + processContext = processContext.forWindowedValue(nextValue); accumulator = combineFnRunner.createAccumulator(key, processContext); accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } // emit the final accumulator diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index f03910bf425c..4434cf8726e1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; import com.google.common.base.Preconditions; @@ -113,6 +115,11 @@ public String toString() { @Override public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { - return new CoderComparator<>(coder); + WindowedValue.WindowedValueCoder windowCoder = (WindowedValue.WindowedValueCoder) coder; + if (windowCoder.getValueCoder() instanceof KvCoder) { + return new KvCoderComperator(windowCoder); + } else { + return new CoderComparator<>(coder); + } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java deleted file mode 100644 index 7b48208845fd..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.types; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; - -/** - * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for - * {@link org.apache.beam.sdk.coders.VoidCoder}. We need this because Flink does not - * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead - * that behaves like a {@code null}, hopefully. - */ -public class VoidCoderTypeSerializer extends TypeSerializer { - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public VoidCoderTypeSerializer duplicate() { - return this; - } - - @Override - public VoidValue createInstance() { - return VoidValue.INSTANCE; - } - - @Override - public VoidValue copy(VoidValue from) { - return from; - } - - @Override - public VoidValue copy(VoidValue from, VoidValue reuse) { - return from; - } - - @Override - public int getLength() { - return 0; - } - - @Override - public void serialize(VoidValue record, DataOutputView target) throws IOException { - target.writeByte(1); - } - - @Override - public VoidValue deserialize(DataInputView source) throws IOException { - source.readByte(); - return VoidValue.INSTANCE; - } - - @Override - public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - source.readByte(); - target.writeByte(1); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof VoidCoderTypeSerializer) { - VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj; - return other.canEqual(this); - } else { - return false; - } - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof VoidCoderTypeSerializer; - } - - @Override - public int hashCode() { - return 0; - } - - public static class VoidValue { - private VoidValue() {} - - public static VoidValue INSTANCE = new VoidValue(); - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 3bf566bce762..6b69d547cf12 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -54,7 +53,7 @@ public static KeyedStream>, K> groupStreamByKey(Da @Override public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : + return isKeyVoid ? (K) VoidValue.INSTANCE : value.getValue().getKey(); } @@ -64,4 +63,11 @@ public TypeInformation getProducedType() { } }); } + + // special type to return as key for null key + public static class VoidValue { + private VoidValue() {} + + public static VoidValue INSTANCE = new VoidValue(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index d6aff7d7a4ee..8cd8351021b4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -47,17 +46,11 @@ public FlinkStreamingCreateFunction(List elements, Coder coder) { @Override public void flatMap(IN value, Collector> out) throws Exception { - @SuppressWarnings("unchecked") - OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; for (byte[] element : elements) { ByteArrayInputStream bai = new ByteArrayInputStream(element); OUT outValue = coder.decode(bai, Coder.Context.OUTER); - if (outValue == null) { - out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } else { - out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } + out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } out.close();