From 3cb997342fc6a8230afcbbb84fb1742a53ef1683 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 14:29:30 -0700 Subject: [PATCH 01/25] Add Window.Bound translator to Flink batch This adds a Window.Bound translator that allows only GlobalWindows. It is a temporary measure, but one that brings the Flink batch translator in line with the Beam model - instead of "ignoring" windows, the GBK is a perfectly valid GBK for GlobalWindows. Previously, the SDK's runner test suite would fail due to the lack of a translator - now some of them will fail due to windowing support, but others have a chance. --- .../FlinkBatchTransformTranslators.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) 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..eabc19d89bbf 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 @@ -52,6 +52,12 @@ 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.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext; +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; @@ -78,12 +84,14 @@ 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.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.lang.reflect.Field; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -117,6 +125,8 @@ public class FlinkBatchTransformTranslators { // 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()); @@ -303,6 +313,31 @@ public void translateNode(Write.Bound transform, FlinkBatchTranslationContext } } + public static class WindowBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + @Override + public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { + PValue input = context.getInput(transform); + DataSet inputDataSet = context.getInputDataSet(input); + + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + if (!(windowingStrategy.getWindowFn() instanceof GlobalWindows)) { + throw new UnsupportedOperationException( + String.format( + "In %s: Only %s is currently supported by batch mode; received %s", + FlinkBatchPipelineTranslator.class.getName(), + GlobalWindows.class.getName(), + windowingStrategy.getWindowFn())); + } + + context.setOutputDataSet(context.getOutput(transform), inputDataSet); + } + } + /** * Translates a GroupByKey while ignoring window assignments. Current ignores windows. */ From b7335f9176e72c1dbd8f30f0b770d290e508dd9c Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 13:11:12 -0700 Subject: [PATCH 02/25] 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 093dc3ec186bc16fd070b09c1678a87dd8f6b47e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 14:04:20 -0700 Subject: [PATCH 03/25] 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 6d00cba87a16ff8f2318db87dab23cc09b7d1c20 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 08:26:50 +0200 Subject: [PATCH 04/25] 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 af7f8580024466dd85cca8a2c070fd1db67490d4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 09:38:55 +0200 Subject: [PATCH 05/25] Add hamcrest dep to Flink Runner --- runners/flink/runner/pom.xml | 26 ++++++++++++++++++-------- 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 427e53b98ae5..4e727098ee42 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 @@ -179,7 +189,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all + none 4 true @@ -213,7 +223,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all + none 4 true From 0cc7363f8324a3ee0aee687d6e433d7db34de093 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 13:09:28 +0200 Subject: [PATCH 06/25] Fix Flink Create and GroupByNullKeyTest, Remove Special VoidSerializer --- .../FlinkBatchTransformTranslators.java | 43 +------ .../FlinkStreamingTransformTranslators.java | 57 +-------- .../functions/FlinkCreateFunction.java | 63 ---------- .../types/CoderTypeInformation.java | 8 +- .../types/VoidCoderTypeSerializer.java | 112 ------------------ .../streaming/FlinkAbstractParDoWrapper.java | 18 +-- ...upByKeyWrapper.java => KVKeySelector.java} | 55 ++++----- .../io/FlinkStreamingCreateFunction.java | 65 ---------- .../flink/streaming/GroupByNullKeyTest.java | 93 +++++++-------- 9 files changed, 88 insertions(+), 426 deletions(-) delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java rename runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/{FlinkGroupByKeyWrapper.java => KVKeySelector.java} (55%) delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index eabc19d89bbf..e45e992d3b30 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,7 +19,6 @@ 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.FlinkDoFnFunction; import org.apache.beam.runners.flink.translation.functions.FlinkKeyedListAggregationFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputDoFnFunction; @@ -40,7 +39,6 @@ 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.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -55,8 +53,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -84,14 +80,10 @@ 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.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.lang.reflect.Field; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -99,7 +91,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 { @@ -118,8 +110,6 @@ public class FlinkBatchTransformTranslators { // 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(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); // TODO we're currently ignoring windows here but that has to change in the future @@ -524,37 +514,6 @@ public void translateNode(View.CreatePCollectionView transform, FlinkBatch } } - private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @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); - } - } - - DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); - FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); - FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); - - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } - private static void transformSideInputs(List> sideInputs, MapPartitionOperator outputDataSet, FlinkBatchTranslationContext context) { 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..712d9345be53 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 @@ -23,10 +23,9 @@ import org.apache.beam.runners.flink.translation.types.FlinkCoder; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper; -import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupByKeyWrapper; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundMultiWrapper; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundWrapper; -import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction; +import org.apache.beam.runners.flink.translation.wrappers.streaming.KVKeySelector; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSink; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; @@ -38,7 +37,6 @@ 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.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -62,7 +60,6 @@ import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -74,8 +71,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -99,7 +94,6 @@ public class FlinkStreamingTransformTranslators { // here you can find all the available translators. static { - TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); TRANSLATORS.put(Read.Bounded.class, new BoundedReadSourceTranslator()); TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); @@ -122,47 +116,6 @@ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getT // Transformation Implementations // -------------------------------------------------------------------------------------------- - private static class CreateStreamingTranslator implements - FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Create.Values transform, FlinkStreamingTranslationContext context) { - PCollection output = context.getOutput(transform); - 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 elementCoder = output.getCoder(); - for (OUT element: elements) { - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - try { - elementCoder.encode(element, bao, Coder.Context.OUTER); - serializedElements.add(bao.toByteArray()); - } catch (IOException e) { - throw new RuntimeException("Could not serialize Create elements using Coder: " + e); - } - } - - - DataStream initDataSet = context.getExecutionEnvironment().fromElements(1); - - FlinkStreamingCreateFunction createFunction = - new FlinkStreamingCreateFunction<>(serializedElements, elementCoder); - - WindowedValue.ValueOnlyWindowedValueCoder windowCoder = WindowedValue.getValueOnlyCoder(elementCoder); - TypeInformation> outputType = new CoderTypeInformation<>(windowCoder); - - DataStream> outputDataStream = initDataSet.flatMap(createFunction) - .returns(outputType); - - context.setOutputDataStream(output, outputDataStream); - } - } - - private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); @@ -382,8 +335,8 @@ public void translateNode(GroupByKey transform, FlinkStreamingTranslationC DataStream>> inputDataStream = context.getInputDataStream(input); KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); + KeyedStream>, K> groupByKStream = + KVKeySelector.keyBy(inputDataStream, inputKvCoder); DataStream>>> groupedByKNWstream = FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(), @@ -403,8 +356,8 @@ public void translateNode(Combine.PerKey transform, FlinkStreaming KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); KvCoder outputKvCoder = (KvCoder) context.getOutput(transform).getCoder(); - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); + KeyedStream>, K> groupByKStream = + KVKeySelector.keyBy(inputDataStream, inputKvCoder); Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) transform.getFn(); DataStream>> groupedByKNWstream = 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/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 895ecef1b92e..4093d6ba323c 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,7 +80,9 @@ public int getTotalFields() { @Override public boolean equals(Object o) { - if (this == o) return true; + 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/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/FlinkAbstractParDoWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index bb6ed67cb29f..26532a47f47b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -197,14 +197,14 @@ protected Aggregator createAggreg } protected void checkTimestamp(WindowedValue ref, Instant timestamp) { - if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { - throw new IllegalArgumentException(String.format( - "Cannot output with timestamp %s. Output timestamps must be no earlier than the " - + "timestamp of the current input (%s) minus the allowed skew (%s). See the " - + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", - timestamp, ref.getTimestamp(), - PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); - } +// if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { +// throw new IllegalArgumentException(String.format( +// "Cannot output with timestamp %s. Output timestamps must be no earlier than the " +// + "timestamp of the current input (%s) minus the allowed skew (%s). See the " +// + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", +// timestamp, ref.getTimestamp(), +// PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); +// } } protected WindowedValue makeWindowedValue( @@ -267,4 +267,4 @@ public abstract WindowingInternals windowingInternalsHelper( WindowedValue inElement, Collector> outCollector); -} \ No newline at end of file +} 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/KVKeySelector.java similarity index 55% rename from runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java rename to runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KVKeySelector.java index 3bf566bce762..2fa6f10758fd 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/KVKeySelector.java @@ -18,10 +18,8 @@ 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; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -32,36 +30,39 @@ import org.apache.flink.streaming.api.datastream.KeyedStream; /** - * This class groups the elements by key. It assumes that already the incoming stream - * is composed of [Key,Value] pairs. - * */ -public class FlinkGroupByKeyWrapper { + * {@link KeySelector} for keying a {@link org.apache.beam.sdk.values.PCollection} + * of {@code KV}. + */ +public class KVKeySelector + implements KeySelector>, K>, ResultTypeQueryable { + + private final TypeInformation keyTypeInfo; - /** - * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement - * multiple interfaces. - */ - private interface KeySelectorWithQueryableResultType extends KeySelector>, K>, ResultTypeQueryable { + private KVKeySelector(TypeInformation keyTypeInfo) { + this.keyTypeInfo = keyTypeInfo; } - public static KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { - final Coder keyCoder = inputKvCoder.getKeyCoder(); - final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); - final boolean isKeyVoid = keyCoder instanceof VoidCoder; + @Override + @SuppressWarnings("unchecked") + public K getKey(WindowedValue> value) throws Exception { + K key = value.getValue().getKey(); + // hack, because Flink does not allow null keys + return key != null ? key : (K) new Integer(0); + } - return inputDataStream.keyBy( - new KeySelectorWithQueryableResultType() { + @Override + public TypeInformation getProducedType() { + return keyTypeInfo; + } - @Override - public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : - value.getValue().getKey(); - } + public static KeyedStream>, K> keyBy( + DataStream>> inputDataStream, + KvCoder inputKvCoder) { - @Override - public TypeInformation getProducedType() { - return keyTypeInfo; - } - }); + final Coder keyCoder = inputKvCoder.getKeyCoder(); + final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); + + return inputDataStream.keyBy(new KVKeySelector(keyTypeInfo)); } + } 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 deleted file mode 100644 index d6aff7d7a4ee..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.wrappers.streaming.io; - -import 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; -import org.apache.beam.sdk.util.WindowedValue; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.util.Collector; -import org.joda.time.Instant; - -import java.io.ByteArrayInputStream; -import java.util.List; - -/** - * This flat map function bootstraps from collection elements and turns them into WindowedValues - * (as required by the Flink runner). - */ -public class FlinkStreamingCreateFunction implements FlatMapFunction> { - - private final List elements; - private final Coder coder; - - public FlinkStreamingCreateFunction(List elements, Coder coder) { - this.elements = elements; - this.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.close(); - } -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index 1b55c61bacf3..c8f970588315 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -29,15 +29,18 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; import com.google.common.base.Joiner; +import com.google.common.collect.Lists; import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; import java.io.Serializable; -import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collections; public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { @@ -61,63 +64,51 @@ protected void postSubmit() throws Exception { compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); } - public static class ExtractUserAndTimestamp extends DoFn, String> { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - KV record = c.element(); - int timestamp = record.getKey(); - String userName = record.getValue(); - if (userName != null) { - // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp)); - } - } - } - @Override protected void testProgram() throws Exception { Pipeline p = FlinkTestPipeline.createForStreaming(); - PCollection output = - p.apply(Create.of(Arrays.asList( - KV.of(0, "user1"), - KV.of(1, "user1"), - KV.of(2, "user1"), - KV.of(10, "user2"), - KV.of(1, "user2"), - KV.of(15000, "user2"), - KV.of(12000, "user2"), - KV.of(25000, "user3")))) - .apply(ParDo.of(new ExtractUserAndTimestamp())) - .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) - .triggering(AfterWatermark.pastEndOfWindow()) - .withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()) - - .apply(ParDo.of(new DoFn>() { - @Override - public void processElement(ProcessContext c) throws Exception { - String elem = c.element(); - c.output(KV.of((Void) null, elem)); - } - })) - .apply(GroupByKey.create()) - .apply(ParDo.of(new DoFn>, String>() { - @Override - public void processElement(ProcessContext c) throws Exception { - KV> elem = c.element(); - StringBuilder str = new StringBuilder(); - str.append("k: " + elem.getKey() + " v:"); - for (String v : elem.getValue()) { - str.append(" " + v); - } - c.output(str.toString()); + PCollection output = p + .apply(Create.timestamped( + TimestampedValue.of("user1", new Instant(0)), + TimestampedValue.of("user1", new Instant(1)), + TimestampedValue.of("user1", new Instant(2)), + TimestampedValue.of("user2", new Instant(10)), + TimestampedValue.of("user2", new Instant(1)), + TimestampedValue.of("user2", new Instant(15000)), + TimestampedValue.of("user2", new Instant(12000)), + TimestampedValue.of("user3", new Instant(25000)))) + .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) + .triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) + + .apply(ParDo.of(new DoFn>() { + @Override + public void processElement(ProcessContext c) throws Exception { + String elem = c.element(); + c.output(KV.of(null, elem)); + } + })) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV> elem = c.element(); + ArrayList strings = Lists.newArrayList(elem.getValue()); + Collections.sort(strings); + StringBuilder str = new StringBuilder(); + str.append("k: " + elem.getKey() + " v:"); + for (String v : strings) { + str.append(" " + v); } - })); + c.output(str.toString()); + } + })); + output.apply(TextIO.Write.to(resultPath)); + p.run(); } } From 5fa21dfa04c4d2cd259af01218e686d55be70b5c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 13:31:18 +0200 Subject: [PATCH 07/25] Fix faulty Flink Flatten with empty PCollectionList --- .../FlinkBatchTransformTranslators.java | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 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 e45e992d3b30..631ca9476f02 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 @@ -33,6 +33,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; @@ -63,6 +64,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; @@ -80,6 +82,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; @@ -490,15 +493,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 30818e8f09375bbaa1691962c1a61b7d9206ef86 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 16:39:02 +0200 Subject: [PATCH 08/25] Fix Flink Batch Partial Combine/Combine We're now using a PerKeyCombineFnRunner for all interaction with the CombineFn. This required adding a proper ProcessContext in FlinkReduceFunction and FlinkPartialReduceFunction, along with adding support for side inputs there. --- .../FlinkBatchTransformTranslators.java | 32 ++-- .../functions/FlinkPartialReduceFunction.java | 146 ++++++++++++++++-- .../functions/FlinkReduceFunction.java | 141 ++++++++++++++++- .../streaming/GroupAlsoByWindowTest.java | 3 +- 4 files changed, 291 insertions(+), 31 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 631ca9476f02..fedcd8b27635 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 @@ -40,6 +40,7 @@ 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.CombineFnBase; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -80,6 +81,7 @@ import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.Grouping; import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Collector; @@ -355,21 +357,20 @@ public void translateNode(GroupByKey transform, FlinkBatchTranslationConte private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override + @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(); - Coder accumulatorCoder = - null; + Coder accumulatorCoder; + 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); @@ -377,15 +378,17 @@ public void translateNode(Combine.PerKey transform, FlinkBatchTransla Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(combineFn); // Partially GroupReduce the values into the intermediate format VA (combine) GroupCombineOperator, KV> groupCombine = new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, "GroupCombine: " + transform.getName()); + transformSideInputs(transform.getSideInputs(), groupCombine, context); + // Reduce fully to VO - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(combineFn); TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); @@ -395,6 +398,8 @@ public void translateNode(Combine.PerKey transform, FlinkBatchTransla GroupReduceOperator, KV> outputDataSet = new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + transformSideInputs(transform.getSideInputs(), outputDataSet, context); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } @@ -532,11 +537,12 @@ public void translateNode(View.CreatePCollectionView transform, FlinkBatch } } - 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()); } 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..798ce373ffd3 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,13 +17,32 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.CombineFnBase; +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.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; -import org.apache.flink.api.common.functions.GroupCombineFunction; +import com.google.common.collect.ImmutableList; + +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.Iterator; +import java.util.List; /** * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a @@ -31,31 +50,138 @@ * {@link org.apache.beam.sdk.values.KV} elements VI, extracts the key and emits accumulated * values which have the intermediate format VA. */ -public class FlinkPartialReduceFunction implements GroupCombineFunction, KV> { +public class FlinkPartialReduceFunction + extends RichGroupCombineFunction, KV> { + + private final CombineFnBase.PerKeyCombineFn combineFn; + + private final DoFn>, KV> doFn; - private final Combine.KeyedCombineFn keyedCombineFn; + public FlinkPartialReduceFunction(CombineFnBase.PerKeyCombineFn combineFn) { + this.combineFn = combineFn; - public FlinkPartialReduceFunction(Combine.KeyedCombineFn - keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + // 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 { + ProcessContext processContext = new ProcessContext(doFn, elements, out); + PerKeyCombineFnRunner combineFnRunner = PerKeyCombineFnRunners.create(combineFn); + + 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); + VA accumulator = combineFnRunner.createAccumulator(key, processContext); + + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); - while(iterator.hasNext()) { + while (iterator.hasNext()) { value = iterator.next().getValue(); - accumulator = keyedCombineFn.addInput(key, accumulator, value); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); } out.collect(KV.of(key, accumulator)); } + + private class ProcessContext extends DoFn>, KV>.ProcessContext { + + private final DoFn>, KV> fn; + + private final Collector> collector; + + private final Iterable> element; + + private ProcessContext( + DoFn>, KV> function, + Iterable> element, + Collector> outCollector) { + function.super(); + super.setupDelegateAggregators(); + + this.fn = function; + this.element = element; + this.collector = outCollector; + } + + @Override + public Iterable> element() { + return this.element; + } + + @Override + public Instant timestamp() { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException("Not supported."); + + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public WindowingInternals>, KV> windowingInternals() { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public PipelineOptions getPipelineOptions() { + throw new UnsupportedOperationException("Not supported."); + } + + @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(KV output) { + collector.collect(output); + } + + @Override + public void outputWithTimestamp(KV output, Instant timestamp) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, + Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().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..ed987852c8c8 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,15 +17,32 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.CombineFnBase; +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.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.KV; +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.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.Iterator; +import java.util.List; /** * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a @@ -33,16 +50,31 @@ * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and merges the * accumulators resulting from the PartialReduce which produced the input VA. */ -public class FlinkReduceFunction implements GroupReduceFunction, KV> { +public class FlinkReduceFunction + extends RichGroupReduceFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; + private final CombineFnBase.PerKeyCombineFn combineFn; - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + private final DoFn>, KV> doFn; + + public FlinkReduceFunction(CombineFnBase.PerKeyCombineFn keyedCombineFn) { + this.combineFn = keyedCombineFn; + + // 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 { + + ProcessContext processContext = new ProcessContext(doFn, values, out); + PerKeyCombineFnRunner combineFnRunner = PerKeyCombineFnRunners.create(combineFn); + Iterator> it = values.iterator(); KV current = it.next(); @@ -51,9 +83,104 @@ public void reduce(Iterable> values, Collector> out) throws while (it.hasNext()) { current = it.next(); - keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); + combineFnRunner.mergeAccumulators(k, + ImmutableList.of(accumulator, current.getValue()), + processContext); + } + + out.collect(KV.of(k, combineFnRunner.extractOutput(k, accumulator, processContext))); + } + + private class ProcessContext extends DoFn>, KV>.ProcessContext { + + private final DoFn>, KV> fn; + + private final Collector> collector; + + private final Iterable> element; + + private ProcessContext( + DoFn>, KV> function, + Iterable> element, + Collector> outCollector) { + function.super(); + super.setupDelegateAggregators(); + + this.fn = function; + this.element = element; + this.collector = outCollector; + } + + @Override + public Iterable> element() { + return this.element; + } + + @Override + public Instant timestamp() { + throw new UnsupportedOperationException("Not supported."); } - out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException("Not supported."); + + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public WindowingInternals>, KV> windowingInternals() { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public PipelineOptions getPipelineOptions() { + throw new UnsupportedOperationException("Not supported."); + } + + @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(KV output) { + collector.collect(output); + } + + @Override + public void outputWithTimestamp(KV output, Instant timestamp) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, + Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, wrapper); + return wrapper; + } } } 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(); From e663de12a9e1fe9dde873d0f4b1c4da1d6819e10 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 16:43:31 +0200 Subject: [PATCH 09/25] Disable MaybeEmptyTestITCase This does not work because Flink Batch does not allow sending null elements. This is a pretty deep thing and hard to fix. In an earlier commit I removed the special TypeSerializer for VoidCoder. Before, we got away by always intercepting the VoidCoder and wrapping it in a TypeSerializer that would always emit a VoidValue instead of a proper null. If the user fn reads this, this will not be consistent with how it should behave, however. --- .../runners/flink/MaybeEmptyTestITCase.java | 132 +++++++++--------- 1 file changed, 66 insertions(+), 66 deletions(-) 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..3bbe63ed6f02 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 @@ -1,66 +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.coders.VoidCoder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.Serializable; - -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"); - } - - @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(resultPath)); - p.run(); - } - -} +///* +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES 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.Serializable; +// +//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"); +// } +// +// @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(resultPath)); +// p.run(); +// } +// +//} From abad0377e31235134a735e3a2012199bd98b6b16 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:32:33 -0700 Subject: [PATCH 10/25] Add RequiresFixedWindows test category --- .../capabilities/RequiresFixedWindows.java | 25 +++++++++++++++++++ .../apache/beam/sdk/transforms/ParDoTest.java | 3 ++- 2 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java new file mode 100644 index 000000000000..aa291d485699 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing.capabilities; + +import org.apache.beam.sdk.transforms.windowing.FixedWindows; + +/** + * Category tag indicating the test requires support for {@link FixedWindows}. + */ +public interface RequiresFixedWindows {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 9193267c2800..6c1cd9a0f1ab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows; import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess; import org.apache.beam.sdk.transforms.ParDo.Bound; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -1347,7 +1348,7 @@ public Void apply(Iterable input) { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresFixedWindows.class}) public void testWindowingInStartAndFinishBundle() { Pipeline pipeline = TestPipeline.create(); From b98ee40a9ea32ec1e5f401eacd2300573cea4b7d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:33:05 -0700 Subject: [PATCH 11/25] Exclude RequiresFixedWindows test category from Flink batch tests --- runners/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/pom.xml b/runners/pom.xml index d2d68dfca132..532c8dd5cc13 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -56,6 +56,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows all 4 From 130665531db6725b2e2bae3d718176428a099c76 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:54:41 -0700 Subject: [PATCH 12/25] Remove unused threadCount from integration tests --- runners/flink/runner/pom.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 4e727098ee42..67e2ff6fb596 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -190,7 +190,6 @@ org.apache.beam.sdk.testing.RunnableOnService none - 4 true org.apache.beam:java-sdk-all @@ -224,7 +223,6 @@ org.apache.beam.sdk.testing.RunnableOnService none - 4 true org.apache.beam:java-sdk-all From c3f71e2742a5a0b15601b35f1c15164e14d068d6 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:55:16 -0700 Subject: [PATCH 13/25] 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 67e2ff6fb596..f5d05dfd78ff 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -221,6 +221,7 @@ test + true org.apache.beam.sdk.testing.RunnableOnService none true From a0376fffea1b363b07adbe25bc87a75a09a98cd5 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 11:19:33 -0700 Subject: [PATCH 14/25] Change PAssert's dummy inputs from (Void) null to integer 0 --- .../org/apache/beam/sdk/testing/PAssert.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 1265acdd9bd4..c2cd598cce0f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Aggregator; @@ -593,7 +593,7 @@ public PDone apply(PBegin input) { final PCollectionView actual = input.apply("CreateActual", createActual); input - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply(Create.of(0).withCoder(VarIntCoder.of())) .apply(ParDo.named("RunChecks").withSideInputs(actual) .of(new CheckerDoFn<>(checkerFn, actual))); @@ -604,8 +604,11 @@ public PDone apply(PBegin input) { /** * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of * a {@link PCollectionView}, and adjusts counters and thrown exceptions for use in testing. + * + *

The input is ignored, but is {@link Integer} to be usable on runners that do not support + * null values. */ - private static class CheckerDoFn extends DoFn { + private static class CheckerDoFn extends DoFn { private final SerializableFunction checkerFn; private final Aggregator success = createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn()); @@ -669,14 +672,17 @@ public PDone apply(PBegin input) { final PCollectionView expected = input.apply("CreateExpected", createExpected); input - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) - .apply(ParDo.named("RunChecks").withSideInputs(actual, expected) + .apply(Create.of(0).withCoder(VarIntCoder.of())) + .apply("RunChecks", ParDo.withSideInputs(actual, expected) .of(new CheckerDoFn<>(relation, actual, expected))); return PDone.in(input.getPipeline()); } - private static class CheckerDoFn extends DoFn { + /** + * Input is ignored, but is {@link Integer} for runners that do not support null values. + */ + private static class CheckerDoFn extends DoFn { private final Aggregator success = createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn()); private final Aggregator failure = From d60229d7dc79b42e721fe6494939cd8fc3eeefa7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 12:49:55 -0700 Subject: [PATCH 15/25] 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 054a42cbaf6c437facfe958f8a2e5dadb59d3e9a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 9 May 2016 12:40:27 +0200 Subject: [PATCH 16/25] Use Int instead of Void in Combine.globally default insertion The single null value is only used as a dummy, thus can also be an integer. This makes it work with runners that don't support sending null values. --- .../main/java/org/apache/beam/sdk/transforms/Combine.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 8a39c98a16e7..1eec2dc5f17c 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 @@ -1398,11 +1398,11 @@ private PCollection insertDefaultValueIfEmpty(PCollection mayb final OutputT defaultValue = fn.defaultValue(); PCollection defaultIfEmpty = maybeEmpty.getPipeline() - .apply("CreateVoid", Create.of((Void) null).withCoder(VoidCoder.of())) + .apply("CreateVoid", Create.of(1).withCoder(VarIntCoder.of())) .apply(ParDo.named("ProduceDefault").withSideInputs(maybeEmptyView).of( - new DoFn() { + new DoFn() { @Override - public void processElement(DoFn.ProcessContext c) { + public void processElement(DoFn.ProcessContext c) { Iterator combined = c.sideInput(maybeEmptyView).iterator(); if (!combined.hasNext()) { c.output(defaultValue); From 8f7f9fbec1a23b098973d15cb60ee3bc55749c2f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 9 May 2016 14:07:57 +0200 Subject: [PATCH 17/25] Fix accumulator update in Flink Reduce Function --- .../flink/translation/functions/FlinkReduceFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ed987852c8c8..36485cb06928 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 @@ -83,7 +83,7 @@ public void reduce(Iterable> values, Collector> out) throws while (it.hasNext()) { current = it.next(); - combineFnRunner.mergeAccumulators(k, + accumulator = combineFnRunner.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()), processContext); } From 7abf64badb0c9c04b21ed91a96c15200d8b4513c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 9 May 2016 14:08:22 +0200 Subject: [PATCH 18/25] Use Int instead of Void as dummy key in Combine.globally This makes it work with runners that don't support sending null values. --- .../java/org/apache/beam/sdk/transforms/Combine.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 1eec2dc5f17c..683b6a0b7ddc 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 @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; @@ -1354,17 +1353,17 @@ public Globally withSideInputs( @Override public PCollection apply(PCollection input) { - PCollection> withKeys = input - .apply(WithKeys.of((Void) null)) - .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); + PCollection> withKeys = input + .apply(WithKeys.of(1)) + .setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder())); - Combine.PerKey combine = + Combine.PerKey combine = Combine.fewKeys(fn.asKeyedFn(), fnDisplayData); if (!sideInputs.isEmpty()) { combine = combine.withSideInputs(sideInputs); } - PCollection> combined; + PCollection> combined; if (fanout >= 2) { combined = withKeys.apply(combine.withHotKeyFanout(fanout)); } else { From 6023361838363640815af370aa13572be9571b9c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 9 May 2016 14:56:36 +0200 Subject: [PATCH 19/25] Use Int instead of Void in Sample The single null value is only used as a dummy, thus can also be an integer. This makes it work with runners that don't support sending null values. --- .../main/java/org/apache/beam/sdk/transforms/Sample.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java index 58188def4226..9c221dcfdea1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; @@ -148,7 +148,7 @@ public PCollection apply(PCollection in) { PCollectionView> iterableView = in.apply(View.asIterable()); return in.getPipeline() - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply(Create.of(1).withCoder(VarIntCoder.of())) .apply(ParDo .withSideInputs(iterableView) .of(new SampleAnyDoFn<>(limit, iterableView))) @@ -165,7 +165,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * A {@link DoFn} that returns up to limit elements from the side input PCollection. */ - private static class SampleAnyDoFn extends DoFn { + private static class SampleAnyDoFn extends DoFn { long limit; final PCollectionView> iterableView; From fa0f7692e86ffc8057fd6559aa16ca7a89d567fb Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 9 May 2016 14:57:26 +0200 Subject: [PATCH 20/25] Use Int instead of Void in FlattenTest The single null value is only used as a dummy, thus can also be an integer. This makes it work with runners that don't support sending null values. --- .../java/org/apache/beam/sdk/transforms/FlattenTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 042b864b2678..8bab756ec882 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -128,8 +129,8 @@ public void testEmptyFlattenAsSideInput() { .apply(View.asIterable()); PCollection output = p - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) - .apply(ParDo.withSideInputs(view).of(new DoFn() { + .apply(Create.of(1).withCoder(VarIntCoder.of())) + .apply(ParDo.withSideInputs(view).of(new DoFn() { @Override public void processElement(ProcessContext c) { for (String side : c.sideInput(view)) { From 19c4cd3332597462c1c6036d0ce2f4429f3bf408 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 9 May 2016 09:51:39 -0700 Subject: [PATCH 21/25] Add RequiresTimestampControl category and tag some tests --- .../RequiresTimestampControl.java | 24 +++++++++++++++++++ .../apache/beam/sdk/io/CountingInputTest.java | 4 +++- .../beam/sdk/io/CountingSourceTest.java | 3 ++- .../beam/sdk/transforms/CreateTest.java | 5 ++-- 4 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java new file mode 100644 index 000000000000..f19e7c4ba8d3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing.capabilities; + +/** + * Category tag indicating the test requires support for controlling timestamps + * of elements. + */ +public interface RequiresTimestampControl {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java index 8c87c2632655..1e6cb07d2540 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -26,6 +27,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; @@ -125,7 +127,7 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testUnboundedInputTimestamps() { Pipeline p = TestPipeline.create(); long numElements = 1000; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java index a261fb274459..8840df5860a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; @@ -136,7 +137,7 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testUnboundedSourceTimestamps() { Pipeline p = TestPipeline.create(); long numElements = 1000; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index e491feadd2d4..69879c6d8624 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Create.Values.CreateSource; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; @@ -233,7 +234,7 @@ public void processElement(ProcessContext c) { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testCreateTimestamped() { Pipeline p = TestPipeline.create(); @@ -252,7 +253,7 @@ public void testCreateTimestamped() { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testCreateTimestampedEmpty() { Pipeline p = TestPipeline.create(); From 55efae45936d602ec1fc4ed47551762a82bc4e91 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 9 May 2016 09:58:19 -0700 Subject: [PATCH 22/25] Exclude groups from Flink batch integration tests --- runners/flink/runner/pom.xml | 4 ++++ runners/pom.xml | 1 - 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index f5d05dfd78ff..c2487a021761 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -189,6 +189,10 @@ org.apache.beam.sdk.testing.RunnableOnService + + org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows, + org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl + none true diff --git a/runners/pom.xml b/runners/pom.xml index 532c8dd5cc13..d2d68dfca132 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -56,7 +56,6 @@ org.apache.beam.sdk.testing.RunnableOnService - org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows all 4 From 99fe94c36ac55cae30ff68e7a7a00180ed69e72a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 9 May 2016 11:08:57 -0700 Subject: [PATCH 23/25] fixup! fix checkstyle of FlattenTest --- .../test/java/org/apache/beam/sdk/transforms/FlattenTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 8bab756ec882..fb5fab5b44e5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; From 1ee56853e025753cc32b6a76d0551f22b387980a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 10 May 2016 13:53:03 +0200 Subject: [PATCH 24/25] [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/examples/WordCount.java | 26 +- runners/flink/runner/pom.xml | 14 - .../FlinkBatchPipelineTranslator.java | 31 +- .../FlinkBatchTransformTranslators.java | 779 +++++++++++------- .../FlinkBatchTranslationContext.java | 56 +- .../FlinkStreamingTransformTranslators.java | 23 +- .../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 | 247 +++--- .../functions/FlinkProcessContext.java | 324 ++++++++ .../functions/FlinkReduceFunction.java | 255 +++--- .../functions/SideInputInitializer.java | 75 ++ .../translation/functions/UnionCoder.java | 152 ---- .../types/CoderTypeInformation.java | 4 +- .../types/CoderTypeSerializer.java | 14 +- .../translation/types/KvCoderComperator.java | 129 +-- .../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 | 139 ++-- .../beam/sdk/transforms/join/UnionCoder.java | 2 +- 34 files changed, 2423 insertions(+), 1423 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/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index 702fb63ec68d..f060f2b02008 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -27,14 +27,21 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; public class WordCount { @@ -109,10 +116,25 @@ public static void main(String[] args) { Pipeline p = Pipeline.create(options); - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + p.apply(Create.timestamped( + TimestampedValue.of("A", new Instant(0)), + TimestampedValue.of("A", new Instant(1)), + TimestampedValue.of("B C D", new Instant(2)), + TimestampedValue.of("B C D", new Instant(11)), + TimestampedValue.of("B C D", new Instant(12)), + TimestampedValue.of("B C D", new Instant(13)), + TimestampedValue.of("B C D", new Instant(14)), + TimestampedValue.of("B C D", new Instant(14)), + TimestampedValue.of("A A", new Instant(0)))) + .apply(Window.into(FixedWindows.of(Duration.millis(5)))) .apply(new CountWords()) .apply(MapElements.via(new FormatAsTextFn())) - .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + System.out.println("SEEING: " + c.window() + ": " + c.element()); + } + })); p.run(); } diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index c2487a021761..18e81d259b74 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -189,10 +189,6 @@ org.apache.beam.sdk.testing.RunnableOnService - - org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows, - org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl - none true @@ -206,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 fedcd8b27635..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,22 +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.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; @@ -47,14 +48,14 @@ 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.GlobalWindows; +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; @@ -62,24 +63,23 @@ 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; @@ -89,14 +89,15 @@ import org.slf4j.LoggerFactory; 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 { @@ -105,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(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; @@ -226,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); @@ -235,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(); @@ -274,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); @@ -287,116 +266,323 @@ 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()); - } - } - private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + TypeInformation> resultTypeInfo = + context.getTypeInfo(context.getOutput(transform)); - @Override - public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { - String name = transform.getName(); - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + DataSet> inputDataSet = context.getInputDataSet(input); + + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowFn windowFn = windowingStrategy.getWindowFn(); - inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); + FlinkAssignWindows assignWindowsFunction = + new FlinkAssignWindows<>(windowFn); + + DataSet> resultDataSet = inputDataSet + .flatMap(assignWindowsFunction) + .name(context.getOutput(transform).getName()) + .returns(resultTypeInfo); + + context.setOutputDataSet(context.getOutput(transform), resultDataSet); } } - public static class WindowBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class GroupByKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + public void translateNode( + GroupByKey transform, + FlinkBatchTranslationContext context) { - @SuppressWarnings("unchecked") - final WindowingStrategy windowingStrategy = - (WindowingStrategy) - context.getOutput(transform).getWindowingStrategy(); + // 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(); - if (!(windowingStrategy.getWindowFn() instanceof GlobalWindows)) { - throw new UnsupportedOperationException( - String.format( - "In %s: Only %s is currently supported by batch mode; received %s", - FlinkBatchPipelineTranslator.class.getName(), - GlobalWindows.class.getName(), - windowingStrategy.getWindowFn())); + 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); } - context.setOutputDataSet(context.getOutput(transform), inputDataSet); + 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); + } } /** - * 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 @SuppressWarnings("unchecked") - public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + Combine.PerKey transform, + FlinkBatchTranslationContext context) { + DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - CombineFnBase.PerKeyCombineFn combineFn = (CombineFnBase.PerKeyCombineFn) 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; try { - accumulatorCoder = combineFn.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) { 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<>(combineFn); + 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()); - transformSideInputs(transform.getSideInputs(), groupCombine, context); + 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<>(combineFn); + // 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); @@ -404,59 +590,68 @@ public void translateNode(Combine.PerKey transform, FlinkBatchTransla } } -// 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()) { @@ -465,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 { @@ -524,15 +779,33 @@ public void flatMap(String s, Collector collector) throws Exception { } } } + + // 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 CreatePCollectionViewTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class CreatePCollectionViewTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + View.CreatePCollectionView> { + @Override - public void translateNode(View.CreatePCollectionView transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - PCollectionView input = transform.apply(null); + public void translateNode( + View.CreatePCollectionView transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + + PCollectionView input = transform.getView(); + context.setSideInputDataSet(input, inputDataSet); } } @@ -548,49 +821,5 @@ private static void transformSideInputs( } } -// 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 712d9345be53..60a9607a9bb4 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; @@ -43,6 +42,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; @@ -60,6 +60,7 @@ import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -181,29 +182,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 798ce373ffd3..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,51 +17,68 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; 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.CombineFnBase; 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.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.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableList; +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.List; +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 - extends RichGroupCombineFunction, KV> { +public class FlinkPartialReduceFunction + extends RichGroupCombineFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; - private final CombineFnBase.PerKeyCombineFn combineFn; + protected final WindowingStrategy windowingStrategy; - private final DoFn>, KV> doFn; + protected final SerializedPipelineOptions serializedOptions; + + protected final Map, WindowingStrategy> sideInputs; + + public FlinkPartialReduceFunction( + CombineFnBase.PerKeyCombineFn combineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { - public FlinkPartialReduceFunction(CombineFnBase.PerKeyCombineFn combineFn) { 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>() { + this.doFn = new DoFn, KV>() { @Override public void processElement(ProcessContext c) throws Exception { @@ -70,118 +87,104 @@ public void processElement(ProcessContext c) throws Exception { } @Override - public void combine(Iterable> elements, Collector> out) throws Exception { - - ProcessContext processContext = new ProcessContext(doFn, elements, out); - PerKeyCombineFnRunner combineFnRunner = PerKeyCombineFnRunners.create(combineFn); + 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 = combineFnRunner.createAccumulator(key, processContext); - - accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + 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()) { - value = iterator.next().getValue(); - accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); - } - - out.collect(KV.of(key, accumulator)); - } - - private class ProcessContext extends DoFn>, KV>.ProcessContext { - - private final DoFn>, KV> fn; - - private final Collector> collector; - - private final Iterable> element; - - private ProcessContext( - DoFn>, KV> function, - Iterable> element, - Collector> outCollector) { - function.super(); - super.setupDelegateAggregators(); - - this.fn = function; - this.element = element; - this.collector = outCollector; - } - - @Override - public Iterable> element() { - return this.element; - } - - @Override - public Instant timestamp() { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public BoundedWindow window() { - throw new UnsupportedOperationException("Not supported."); - - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } - - @Override - public WindowingInternals>, KV> windowingInternals() { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public PipelineOptions getPipelineOptions() { - throw new UnsupportedOperationException("Not supported."); - } - - @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())); + 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); } - return view.fromIterableInternal(windowedValueList); - } - - @Override - public void output(KV output) { - collector.collect(output); - } - @Override - public void outputWithTimestamp(KV output, Instant timestamp) { - throw new UnsupportedOperationException("Not supported."); + // 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; } - @Override - public void sideOutput(TupleTag tag, T output) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, - Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = - new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } + // 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 36485cb06928..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,51 +17,72 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; 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.CombineFnBase; 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.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.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; +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.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 - extends RichGroupReduceFunction, KV> { +public class FlinkReduceFunction + extends RichGroupReduceFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; - private final CombineFnBase.PerKeyCombineFn combineFn; + protected final WindowingStrategy windowingStrategy; - private final DoFn>, KV> doFn; + protected final Map, WindowingStrategy> sideInputs; + + protected final SerializedPipelineOptions serializedOptions; + + public FlinkReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { - public FlinkReduceFunction(CombineFnBase.PerKeyCombineFn 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>() { + this.doFn = new DoFn, KV>() { @Override public void processElement(ProcessContext c) throws Exception { @@ -70,117 +91,105 @@ public void processElement(ProcessContext c) throws Exception { } @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - - ProcessContext processContext = new ProcessContext(doFn, values, out); - PerKeyCombineFnRunner combineFnRunner = PerKeyCombineFnRunners.create(combineFn); - - Iterator> it = values.iterator(); - - KV current = it.next(); - K k = current.getKey(); - VA accumulator = current.getValue(); - - while (it.hasNext()) { - current = it.next(); - accumulator = combineFnRunner.mergeAccumulators(k, - ImmutableList.of(accumulator, current.getValue()), - processContext); - } - - out.collect(KV.of(k, combineFnRunner.extractOutput(k, accumulator, processContext))); - } - - private class ProcessContext extends DoFn>, KV>.ProcessContext { - - private final DoFn>, KV> fn; - - private final Collector> collector; - - private final Iterable> element; - - private ProcessContext( - DoFn>, KV> function, - Iterable> element, - Collector> outCollector) { - function.super(); - super.setupDelegateAggregators(); - - this.fn = function; - this.element = element; - this.collector = outCollector; - } - - @Override - public Iterable> element() { - return this.element; - } - - @Override - public Instant timestamp() { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public BoundedWindow window() { - throw new UnsupportedOperationException("Not supported."); - - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } - - @Override - public WindowingInternals>, KV> windowingInternals() { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public PipelineOptions getPipelineOptions() { - throw new UnsupportedOperationException("Not supported."); - } - - @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())); + 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); } - return view.fromIterableInternal(windowedValueList); - - } - - @Override - public void output(KV output) { - collector.collect(output); - } - - @Override - public void outputWithTimestamp(KV output, Instant timestamp) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - throw new UnsupportedOperationException("Not supported."); } + 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()); + } - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - throw new UnsupportedOperationException("Not supported."); + // 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; } - @Override - protected Aggregator createAggregatorInternal( - String name, - Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = - new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } + // 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 4093d6ba323c..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 @@ -83,7 +83,9 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) return false; + 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..25fe73d54a27 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,46 @@ 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); + + +// inputWrapper.setInputView(firstSource); +// K firstKey = keyCoder.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); +// } } @Override public boolean supportsNormalizedKey() { - return true; + return false; } @Override @@ -209,12 +220,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 +248,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 +267,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 3bbe63ed6f02..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 @@ -1,66 +1,73 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -//package org.apache.beam.runners.flink; -// -//import org.apache.beam.sdk.Pipeline; -//import org.apache.beam.sdk.coders.VoidCoder; -//import org.apache.beam.sdk.io.TextIO; -//import org.apache.beam.sdk.transforms.Create; -//import org.apache.beam.sdk.transforms.DoFn; -//import org.apache.beam.sdk.transforms.ParDo; -// -//import org.apache.flink.test.util.JavaProgramTestBase; -// -//import java.io.Serializable; -// -//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"); -// } -// -// @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(resultPath)); -// p.run(); -// } -// -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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/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 a2c988cfd9d6f80f06b60777c1a2e6a494233410 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 12 May 2016 18:29:39 +0200 Subject: [PATCH 25/25] fixup, remove WordCount changes, remove KvCoderComparator code --- .../runners/flink/examples/WordCount.java | 26 ++---------------- .../translation/types/KvCoderComperator.java | 27 ------------------- 2 files changed, 2 insertions(+), 51 deletions(-) diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index f060f2b02008..702fb63ec68d 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -27,21 +27,14 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; public class WordCount { @@ -116,25 +109,10 @@ public static void main(String[] args) { Pipeline p = Pipeline.create(options); - p.apply(Create.timestamped( - TimestampedValue.of("A", new Instant(0)), - TimestampedValue.of("A", new Instant(1)), - TimestampedValue.of("B C D", new Instant(2)), - TimestampedValue.of("B C D", new Instant(11)), - TimestampedValue.of("B C D", new Instant(12)), - TimestampedValue.of("B C D", new Instant(13)), - TimestampedValue.of("B C D", new Instant(14)), - TimestampedValue.of("B C D", new Instant(14)), - TimestampedValue.of("A A", new Instant(0)))) - .apply(Window.into(FixedWindows.of(Duration.millis(5)))) + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) .apply(new CountWords()) .apply(MapElements.via(new FormatAsTextFn())) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - System.out.println("SEEING: " + c.window() + ": " + c.element()); - } - })); + .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); p.run(); } 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 25fe73d54a27..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 @@ -170,33 +170,6 @@ public int compareSerialized( inputWrapper.setInputView(secondSource); WindowedValue> second = coder.decode(inputWrapper, Coder.Context.NESTED); return compare(first, second); - - -// inputWrapper.setInputView(firstSource); -// K firstKey = keyCoder.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); -// } } @Override