From fa8bc7cd3bea3dac0484b0af139248e1f39d0317 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 13:11:12 -0700 Subject: [PATCH 01/14] 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 3edf6f30c22d..b5ffced60d19 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 @@ -108,7 +108,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { this.flinkJobEnv.translate(pipeline); LOG.info("Starting execution of Flink program."); - + JobExecutionResult result; try { result = this.flinkJobEnv.executePipeline(); @@ -138,20 +138,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 3e76a85ec23da83c5ffc734c80eee882be1c3f38 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 2 May 2016 14:04:20 -0700 Subject: [PATCH 02/14] 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 | 106 ++++++++++++++++++++++++++--------- 1 file changed, 79 insertions(+), 27 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index a53a386c2828..cde910873285 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,7 +62,8 @@ flink-avro_2.10 ${flink.version} - + + org.apache.beam java-sdk-all @@ -111,6 +87,21 @@ + + + + org.apache.beam + java-sdk-all + tests + test + + + org.slf4j + slf4j-jdk14 + + + + org.apache.beam java-examples-all @@ -168,10 +159,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 60f75655b5788b400a81abda32410eaa784f0aad Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:54:41 -0700 Subject: [PATCH 03/14] Remove unused threadCount from integration tests --- runners/flink/runner/pom.xml | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index cde910873285..f94ce6894203 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -168,8 +168,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all - 4 + none true org.apache.beam:java-sdk-all @@ -202,8 +201,7 @@ org.apache.beam.sdk.testing.RunnableOnService - all - 4 + none true org.apache.beam:java-sdk-all From 8ec74ac46c10c6f1fd37d5a6049d9cb03008c432 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 10:55:16 -0700 Subject: [PATCH 04/14] 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 f94ce6894203..7e8c5a98a79d 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -200,6 +200,7 @@ test + true org.apache.beam.sdk.testing.RunnableOnService none true From edef0ce1dcf1405e037c23f997cb1c7980e584f0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 6 May 2016 12:49:55 -0700 Subject: [PATCH 05/14] 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 f0899a682c19231f59f539e9699087ad443734cc Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 09:38:55 +0200 Subject: [PATCH 06/14] Add hamcrest dependency to Flink Runner Without it the RunnableOnService tests seem to not work --- runners/flink/runner/pom.xml | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 7e8c5a98a79d..fda27a863581 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -88,6 +88,22 @@ + + org.hamcrest + hamcrest-all + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + org.apache.beam @@ -124,12 +140,6 @@ org.apache.flink flink-test-utils_2.10 ${flink.version} - test - - - org.mockito - mockito-all - test From 84e10a4b0cbc78949f073737948de00bdb8179f3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 May 2016 08:26:50 +0200 Subject: [PATCH 07/14] 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 3d39e8182cab..512b8229c9ce 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; @@ -47,6 +50,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 19d4e564dba1a595c164393dc525496485632e28 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 13 May 2016 14:17:50 +0200 Subject: [PATCH 08/14] Fix faulty Flink Flatten when PCollectionList is empty --- .../FlinkBatchTransformTranslators.java | 32 +++++++++++++++---- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index a03352efae15..07785aa47c69 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Read; @@ -61,6 +62,7 @@ import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -78,6 +80,7 @@ import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,7 +94,7 @@ /** * Translators for transforming * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to - * Flink {@link org.apache.flink.api.java.DataSet}s + * Flink {@link org.apache.flink.api.java.DataSet}s. */ public class FlinkBatchTransformTranslators { @@ -465,15 +468,30 @@ public void translateNode(ParDo.BoundMulti transform, FlinkBatchTransla private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override + @SuppressWarnings("unchecked") public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { List> allInputs = context.getInput(transform).getAll(); DataSet result = null; - for(PCollection collection : allInputs) { - DataSet current = context.getInputDataSet(collection); - if (result == null) { - result = current; - } else { - result = result.union(current); + if (allInputs.isEmpty()) { + // create an empty dummy source to satisfy downstream operations + // we cannot create an empty source in Flink, therefore we have to + // add the flatMap that simply never forwards the single element + DataSource dummySource = + context.getExecutionEnvironment().fromElements("dummy"); + result = dummySource.flatMap(new FlatMapFunction() { + @Override + public void flatMap(String s, Collector collector) throws Exception { + // never return anything + } + }).returns(new CoderTypeInformation<>((Coder) VoidCoder.of())); + } else { + for (PCollection collection : allInputs) { + DataSet current = context.getInputDataSet(collection); + if (result == null) { + result = current; + } else { + result = result.union(current); + } } } context.setOutputDataSet(context.getOutput(transform), result); From eadabd58f0ffca76535bbc681f051bf4292f69d9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 10 May 2016 13:53:03 +0200 Subject: [PATCH 09/14] [BEAM-270] Support Timestamps/Windows in Flink Batch With this change we always use WindowedValue for the underlying Flink DataSets instead of just T. This allows us to support windowing as well. This changes also a lot of other stuff enabled by the above: - Use WindowedValue throughout - Add proper translation for Window.into() - Make side inputs window aware - Make GroupByKey and Combine transformations window aware, this includes support for merging windows. GroupByKey is implemented as a Combine with a concatenating CombineFn, for simplicity This removes Flink specific transformations for things that are handled by builtin sources/sinks, among other things this: - Removes special translation for AvroIO.Read/Write and TextIO.Read/Write - Removes special support for Write.Bound, this was not working properly and is now handled by the Beam machinery that uses DoFns for this - Removes special translation for binary Co-Group, the code was still in there but was never used With this change all RunnableOnService tests run on Flink Batch. --- runners/flink/runner/pom.xml | 10 - .../FlinkBatchPipelineTranslator.java | 4 +- .../FlinkBatchTransformTranslators.java | 821 +++++++++++------- .../FlinkBatchTranslationContext.java | 56 +- .../FlinkStreamingTransformTranslators.java | 22 +- .../FlinkStreamingTranslationContext.java | 29 +- .../functions/FlinkAssignContext.java | 56 ++ .../functions/FlinkAssignWindows.java | 51 ++ .../FlinkCoGroupKeyedListAggregator.java | 61 -- .../functions/FlinkDoFnFunction.java | 194 ++--- .../FlinkKeyedListAggregationFunction.java | 78 -- .../FlinkMergingPartialReduceFunction.java | 211 +++++ .../functions/FlinkMergingReduceFunction.java | 215 +++++ .../FlinkMultiOutputDoFnFunction.java | 157 ++-- .../FlinkMultiOutputProcessContext.java | 176 ++++ .../FlinkMultiOutputPruningFunction.java | 25 +- .../FlinkNoElementAssignContext.java | 71 ++ .../functions/FlinkPartialReduceFunction.java | 175 +++- .../functions/FlinkProcessContext.java | 324 +++++++ .../functions/FlinkReduceFunction.java | 174 +++- .../functions/SideInputInitializer.java | 75 ++ .../translation/functions/UnionCoder.java | 152 ---- .../types/CoderTypeInformation.java | 12 +- .../types/CoderTypeSerializer.java | 14 +- .../translation/types/KvCoderComperator.java | 102 +-- .../types/KvCoderTypeInformation.java | 63 +- .../wrappers/CombineFnAggregatorWrapper.java | 94 -- .../SerializableFnAggregatorWrapper.java | 31 +- .../wrappers/SinkOutputFormat.java | 10 +- .../wrappers/SourceInputFormat.java | 18 +- .../apache/beam/runners/flink/AvroITCase.java | 36 +- .../runners/flink/MaybeEmptyTestITCase.java | 9 +- .../beam/sdk/transforms/join/UnionCoder.java | 2 +- 33 files changed, 2361 insertions(+), 1167 deletions(-) create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index fda27a863581..b29a5bf221c0 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -191,16 +191,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 512b8229c9ce..69c02a22b36d 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 { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index 07785aa47c69..fda4d29556f3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -17,23 +17,23 @@ */ package org.apache.beam.runners.flink.translation; -import org.apache.beam.runners.flink.io.ConsoleIO; -import org.apache.beam.runners.flink.translation.functions.FlinkCoGroupKeyedListAggregator; -import org.apache.beam.runners.flink.translation.functions.FlinkCreateFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; -import org.apache.beam.runners.flink.translation.functions.FlinkKeyedListAggregationFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMergingPartialReduceFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMergingReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputDoFnFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction; import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction; -import org.apache.beam.runners.flink.translation.functions.UnionCoder; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; @@ -41,60 +41,63 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGbkResultSchema; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.join.UnionCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.operators.CoGroupOperator; import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.api.java.operators.DataSource; import org.apache.flink.api.java.operators.FlatMapOperator; import org.apache.flink.api.java.operators.GroupCombineOperator; import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.Grouping; +import org.apache.flink.api.java.operators.MapOperator; import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; /** - * Translators for transforming - * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to - * Flink {@link org.apache.flink.api.java.DataSet}s. + * Translators for transforming {@link PTransform PTransforms} to + * Flink {@link DataSet DataSets}. */ public class FlinkBatchTransformTranslators { @@ -103,113 +106,90 @@ public class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); + private static final Map< + Class, + FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); - // register the known translators static { TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); - // we don't need this because we translate the Combine.PerKey directly - //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); - - TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); - // TODO we're currently ignoring windows here but that has to change in the future - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); + TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslatorBatch()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); - - TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch()); - - TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch()); - TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); - TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch()); - - TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch()); - TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch()); - - // Flink-specific - TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch()); - } - public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator(PTransform transform) { + public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator( + PTransform transform) { return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class ReadSourceTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override public void translateNode(Read.Bounded transform, FlinkBatchTranslationContext context) { String name = transform.getName(); BoundedSource source = transform.getSource(); PCollection output = context.getOutput(transform); - Coder coder = output.getCoder(); - TypeInformation typeInformation = context.getTypeInfo(output); + TypeInformation> typeInformation = context.getTypeInfo(output); - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), - new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); + DataSource> dataSource = new DataSource<>( + context.getExecutionEnvironment(), + new SourceInputFormat<>(source, context.getPipelineOptions()), + typeInformation, + name); context.setOutputDataSet(output, dataSource); } } - private static class AvroIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class); + private static class WriteSinkTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); + public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { String name = transform.getName(); -// Schema schema = transform.getSchema(); - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - - // This is super hacky, but unfortunately we cannot get the type otherwise - Class extractedAvroType; - try { - Field typeField = transform.getClass().getDeclaredField("type"); - typeField.setAccessible(true); - @SuppressWarnings("unchecked") - Class avroType = (Class) typeField.get(transform); - extractedAvroType = avroType; - } catch (NoSuchFieldException | IllegalAccessException e) { - // we know that the field is there and it is accessible - throw new RuntimeException("Could not access type from AvroIO.Bound", e); - } - - DataSource source = new DataSource<>(context.getExecutionEnvironment(), - new AvroInputFormat<>(new Path(path), extractedAvroType), - typeInformation, name); + PValue input = context.getInput(transform); + DataSet> inputDataSet = context.getInputDataSet(input); - context.setOutputDataSet(output, source); + inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())) + .name(name); } } - private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class AvroIOWriteTranslatorBatch implements + FlinkBatchPipelineTranslator.BatchTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class); + @Override - public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + AvroIO.Write.Bound transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); int numShards = transform.getNumShards(); String shardNameTemplate = transform.getShardNameTemplate(); // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", + LOG.warn( + "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + LOG.warn( + "Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", + shardNameTemplate); // This is super hacky, but unfortunately we cannot get the type otherwise Class extractedAvroType; @@ -224,8 +204,17 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation throw new RuntimeException("Could not access type from AvroIO.Bound", e); } - DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path - (filenamePrefix), extractedAvroType)); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(context.getInput(transform).getCoder())); + + + DataSink dataSink = valueStream.output( + new AvroOutputFormat<>(new Path(filenamePrefix), extractedAvroType)); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -233,37 +222,16 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation } } - private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class); - - @Override - public void translateNode(TextIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); - String name = transform.getName(); - - TextIO.CompressionType compressionType = transform.getCompressionType(); - boolean needsValidation = transform.needsValidation(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); - LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); - - context.setOutputDataSet(output, source); - } - } - - private static class TextIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class TextIOWriteTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); @Override - public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode( + TextIO.Write.Bound transform, + FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + DataSet> inputDataSet = context.getInputDataSet(input); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); @@ -272,12 +240,25 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation String shardNameTemplate = transform.getShardNameTemplate(); // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + LOG.warn( + "Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", + needsValidation); + LOG.warn( + "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", + filenameSuffix); + LOG.warn( + "Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", + shardNameTemplate); - //inputDataSet.print(); - DataSink dataSink = inputDataSet.writeAsText(filenamePrefix); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(transform.getCoder())); + + DataSink dataSink = valueStream.writeAsText(filenamePrefix); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -285,148 +266,392 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation } } - private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { + private static class WindowBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + @Override - public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); - inputDataSet.printOnTaskManager(transform.getName()); + + TypeInformation> resultTypeInfo = + context.getTypeInfo(context.getOutput(transform)); + + DataSet> inputDataSet = context.getInputDataSet(input); + + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowFn windowFn = windowingStrategy.getWindowFn(); + + FlinkAssignWindows assignWindowsFunction = + new FlinkAssignWindows<>(windowFn); + + DataSet> resultDataSet = inputDataSet + .flatMap(assignWindowsFunction) + .name(context.getOutput(transform).getName()) + .returns(resultTypeInfo); + + context.setOutputDataSet(context.getOutput(transform), resultDataSet); } } - private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class GroupByKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { - String name = transform.getName(); - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + public void translateNode( + GroupByKey transform, + FlinkBatchTranslationContext context) { + + // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API + // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn + + DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + + Combine.KeyedCombineFn, List> combineFn = + new Concatenate().asKeyedFn(); + + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + + Coder> accumulatorCoder; + + try { + accumulatorCoder = + combineFn.getAccumulatorCoder( + context.getInput(transform).getPipeline().getCoderRegistry(), + inputCoder.getKeyCoder(), + inputCoder.getValueCoder()); + } catch (CannotProvideCoderException e) { + throw new RuntimeException(e); + } + + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + + TypeInformation>> kvCoderTypeInformation = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + inputCoder, + windowingStrategy.getWindowFn().windowCoder())); + + TypeInformation>>> partialReduceTypeInfo = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), + windowingStrategy.getWindowFn().windowCoder())); + + Grouping>> inputGrouping = + new UnsortedGrouping<>( + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, + kvCoderTypeInformation)); + + FlinkPartialReduceFunction, ?> partialReduceFunction; + FlinkReduceFunction, List, ?> reduceFunction; + + if (windowingStrategy.getWindowFn().isNonMerging()) { + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + } else { + if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) { + throw new UnsupportedOperationException( + "Merging WindowFn with windows other than IntervalWindow are not supported."); + } + WindowingStrategy intervalStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkMergingPartialReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkMergingReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + } + + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); + + Grouping>>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format VO + GroupReduceOperator< + WindowedValue>>, WindowedValue>>> outputDataSet = + new GroupReduceOperator<>( + intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName()); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); - inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); } } /** - * Translates a GroupByKey while ignoring window assignments. Current ignores windows. + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

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

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

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

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

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

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

This is different from the pair of function for the non-merging windows case + * in that we cannot do combining before the shuffle because elements would not + * yet be in their correct windows for side-input access. + */ +public class FlinkMergingNonShuffleReduceFunction< + K, InputT, AccumT, OutputT, W extends IntervalWindow> + extends RichGroupReduceFunction>, WindowedValue>> { + + private final CombineFnBase.PerKeyCombineFn combineFn; + + private final DoFn, KV> doFn; + + private final WindowingStrategy windowingStrategy; + + private final Map, WindowingStrategy> sideInputs; + + private final SerializedPipelineOptions serializedOptions; + + public FlinkMergingNonShuffleReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + + this.combineFn = keyedCombineFn; + + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; + } + + @Override + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + List>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // merge windows, we have to do it in an extra pre-processing step and + // can't do it as we go since the window of early elements would not + // be correct when calling the CombineFn + mergeWindow(sortedInput); + + // iterate over the elements that are sorted by window timestamp + final Iterator>> iterator = sortedInput.iterator(); + + // create accumulator using the first elements key + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (currentWindow.equals(nextWindow)) { + // continue accumulating and merge windows + + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + windowTimestamp = outputTimeFn.combine( + windowTimestamp, + outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow)); + + } else { + // emit the value that we currently have + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + + currentWindow = nextWindow; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + } + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + } + + /** + * Merge windows. This assumes that the list of elements is sorted by window-end timestamp. + * This replaces windows in the input list. + */ + private void mergeWindow(List>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> nextValue = elements.get(i); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + if (currentWindow.intersects(nextWindow)) { + // we continue + currentWindow = currentWindow.span(nextWindow); + } else { + // retrofit the merged window to all windows up to "currentStart" + for (int j = i - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + currentStart = i; + currentWindow = nextWindow; + } + } + if (currentStart < elements.size() - 1) { + // we have to retrofit the last batch + for (int j = elements.size() - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java index 47f920770d45..c12e4204a3f0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -85,11 +85,9 @@ public void combine( List>> sortedInput = Lists.newArrayList(); for (WindowedValue> inputValue: elements) { for (WindowedValue> exploded: inputValue.explodeWindows()) { - System.out.println("ADDING EXLODE: " + exploded); sortedInput.add(exploded); } } - System.out.println("SORTED SIZE: " + sortedInput.size()); Collections.sort(sortedInput, new Comparator>>() { @Override public int compare( @@ -148,15 +146,11 @@ public int compare( currentWindow = nextWindow; InputT value = nextValue.getValue().getValue(); - processContext = processContext.forWindowedValue(currentValue); + processContext = processContext.forWindowedValue(nextValue); accumulator = combineFnRunner.createAccumulator(key, processContext); accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } // emit the final accumulator diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java index 3bd08075401f..07d1c9741533 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -137,8 +137,6 @@ public int compare( windowTimestamps.add(nextValue.getTimestamp()); } else { - // emit the value that we currently have - processContext = processContext.forWindowedValue(currentValue); out.collect( WindowedValue.of( KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), @@ -148,20 +146,14 @@ public int compare( windowTimestamps.clear(); + processContext = processContext.forWindowedValue(nextValue); + currentWindow = nextWindow; accumulator = nextValue.getValue().getValue(); windowTimestamps.add(nextValue.getTimestamp()); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } - // if at the end of the iteration we have a change in windows - // the ProcessContext will not have been updated - processContext = processContext.forWindowedValue(currentValue); - // emit the final accumulator out.collect( WindowedValue.of( diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 88308098bee9..c29e1df2ceb0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -168,15 +168,11 @@ public int compare( currentWindow = nextWindow; InputT value = nextValue.getValue().getValue(); - processContext = processContext.forWindowedValue(currentValue); + processContext = processContext.forWindowedValue(nextValue); accumulator = combineFnRunner.createAccumulator(key, processContext); accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); } - - // we have to keep track so that we can set the context to the right - // windowed value when windows change in the iterable - currentValue = nextValue; } // emit the final accumulator diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index f03910bf425c..4434cf8726e1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; import com.google.common.base.Preconditions; @@ -113,6 +115,11 @@ public String toString() { @Override public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { - return new CoderComparator<>(coder); + WindowedValue.WindowedValueCoder windowCoder = (WindowedValue.WindowedValueCoder) coder; + if (windowCoder.getValueCoder() instanceof KvCoder) { + return new KvCoderComperator(windowCoder); + } else { + return new CoderComparator<>(coder); + } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java deleted file mode 100644 index 7b48208845fd..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.types; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; - -/** - * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for - * {@link org.apache.beam.sdk.coders.VoidCoder}. We need this because Flink does not - * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead - * that behaves like a {@code null}, hopefully. - */ -public class VoidCoderTypeSerializer extends TypeSerializer { - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public VoidCoderTypeSerializer duplicate() { - return this; - } - - @Override - public VoidValue createInstance() { - return VoidValue.INSTANCE; - } - - @Override - public VoidValue copy(VoidValue from) { - return from; - } - - @Override - public VoidValue copy(VoidValue from, VoidValue reuse) { - return from; - } - - @Override - public int getLength() { - return 0; - } - - @Override - public void serialize(VoidValue record, DataOutputView target) throws IOException { - target.writeByte(1); - } - - @Override - public VoidValue deserialize(DataInputView source) throws IOException { - source.readByte(); - return VoidValue.INSTANCE; - } - - @Override - public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - source.readByte(); - target.writeByte(1); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof VoidCoderTypeSerializer) { - VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj; - return other.canEqual(this); - } else { - return false; - } - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof VoidCoderTypeSerializer; - } - - @Override - public int hashCode() { - return 0; - } - - public static class VoidValue { - private VoidValue() {} - - public static VoidValue INSTANCE = new VoidValue(); - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 3bf566bce762..6b69d547cf12 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -54,7 +53,7 @@ public static KeyedStream>, K> groupStreamByKey(Da @Override public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : + return isKeyVoid ? (K) VoidValue.INSTANCE : value.getValue().getKey(); } @@ -64,4 +63,11 @@ public TypeInformation getProducedType() { } }); } + + // special type to return as key for null key + public static class VoidValue { + private VoidValue() {} + + public static VoidValue INSTANCE = new VoidValue(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index d6aff7d7a4ee..8cd8351021b4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -47,17 +46,11 @@ public FlinkStreamingCreateFunction(List elements, Coder coder) { @Override public void flatMap(IN value, Collector> out) throws Exception { - @SuppressWarnings("unchecked") - OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; for (byte[] element : elements) { ByteArrayInputStream bai = new ByteArrayInputStream(element); OUT outValue = coder.decode(bai, Coder.Context.OUTER); - if (outValue == null) { - out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } else { - out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } + out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } out.close(); From b5c6b1c8f7408c7f7e174df77e71ca062811800c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 18 May 2016 17:43:36 +0200 Subject: [PATCH 12/14] Remove Flink-Specific ConsoleIO this can be done using a Sysout in a ParDo --- .../beam/runners/flink/io/ConsoleIO.java | 82 ------------------- 1 file changed, 82 deletions(-) delete mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java deleted file mode 100644 index 9c36c217df36..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES 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.io; - -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; - -/** - * Transform for printing the contents of a {@link org.apache.beam.sdk.values.PCollection}. - * to standard output. - * - * This is Flink-specific and will only work when executed using the - * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}. - */ -public class ConsoleIO { - - /** - * A PTransform that writes a PCollection to a standard output. - */ - public static class Write { - - /** - * Returns a ConsoleIO.Write PTransform with a default step name. - */ - public static Bound create() { - return new Bound(); - } - - /** - * Returns a ConsoleIO.Write PTransform with the given step name. - */ - public static Bound named(String name) { - return new Bound().named(name); - } - - /** - * A PTransform that writes a bounded PCollection to standard output. - */ - public static class Bound extends PTransform, PDone> { - private static final long serialVersionUID = 0; - - Bound() { - super("ConsoleIO.Write"); - } - - Bound(String name) { - super(name); - } - - /** - * Returns a new ConsoleIO.Write PTransform that's like this one but with the given - * step - * name. Does not modify this object. - */ - public Bound named(String name) { - return new Bound(name); - } - - @Override - public PDone apply(PCollection input) { - return PDone.in(input.getPipeline()); - } - } - } -} - From aa179ad92bb04bf7732a6c4675b65533417ecab4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 18 May 2016 18:33:07 +0200 Subject: [PATCH 13/14] Fix checkstyle warning (to force Jenkins to rerun) --- .../flink/translation/FlinkBatchTransformTranslators.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 86aee84ab1cb..83588076c46c 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 @@ -558,7 +558,8 @@ public void translateNode( Grouping>> intermediateGrouping = new UnsortedGrouping<>( - groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + groupCombine, + new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); // Fully reduce the values and create output format OutputT GroupReduceOperator< @@ -594,7 +595,8 @@ public void translateNode( Grouping>> grouping = new UnsortedGrouping<>( - inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); // Fully reduce the values and create output format OutputT GroupReduceOperator< From 6c4af76fcc70a8f36acf3645ef7e0fd85d04fefe Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 May 2016 14:05:05 +0200 Subject: [PATCH 14/14] Add surefire plugin to java 8 example tests --- examples/java8/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index b4a9ec6d7fa4..e211739a9412 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -49,6 +49,18 @@ + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + + + + org.apache.maven.plugins maven-dependency-plugin