diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index a1d53703ad70..18e81d259b74 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -34,31 +34,6 @@ jar - - - disable-runnable-on-service-tests - - true - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - runnable-on-service-tests - - true - - - - - - - - - @@ -87,6 +62,7 @@ flink-avro_2.10 ${flink.version} + org.apache.beam @@ -120,7 +96,39 @@ + + + + org.hamcrest + hamcrest-all + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + org.apache.beam + java-sdk-all + tests + test + + + org.slf4j + slf4j-jdk14 + + + + org.apache.beam java-examples-all @@ -143,12 +151,6 @@ org.apache.flink flink-test-utils_2.10 ${flink.version} - test - - - org.mockito - mockito-all - test @@ -178,10 +180,60 @@ org.apache.maven.plugins maven-surefire-plugin + + + runnable-on-service-tests + integration-test + + test + + + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=false" + ] + + + + + + streaming-runnable-on-service-tests + integration-test + + test + + + true + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=true" + ] + + + + + + + - diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index a389d7a076c8..5e44d997e970 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -110,7 +110,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { this.flinkJobEnv.translate(pipeline); LOG.info("Starting execution of Flink program."); - + JobExecutionResult result; try { result = this.flinkJobEnv.executePipeline(); @@ -140,20 +140,6 @@ public FlinkPipelineOptions getPipelineOptions() { return options; } - /** - * Constructs a runner with default properties for testing. - * - * @return The newly created runner. - */ - public static FlinkPipelineRunner createForTest(boolean streaming) { - FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); - // we use [auto] for testing since this will make it pick up the Testing - // ExecutionEnvironment - options.setFlinkMaster("[auto]"); - options.setStreaming(streaming); - return new FlinkPipelineRunner(options); - } - @Override public Output apply( PTransform transform, Input input) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index cd99f4e65bce..ec61805a4ed0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -41,7 +41,9 @@ private FlinkRunnerRegistrar() { } public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of(FlinkPipelineRunner.class); + return ImmutableList.>>of( + FlinkPipelineRunner.class, + TestFlinkPipelineRunner.class); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java new file mode 100644 index 000000000000..139aebf9dd2b --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink; + +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; + +import org.apache.flink.runtime.client.JobExecutionException; + +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) { + 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() { + return delegate.getPipelineOptions(); + } +} + + diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 456cf09457b6..98229ae331f3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -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,13 +25,15 @@ 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; /** - * 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 { @@ -52,6 +55,17 @@ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions op this.batchContext = new FlinkBatchTranslationContext(env, options); } + @Override + @SuppressWarnings("rawtypes, unchecked") + public void translate(Pipeline pipeline) { + super.translate(pipeline); + + // terminate dangling DataSets + for (DataSet dataSet: batchContext.getDanglingDataSets().values()) { + dataSet.output(new DiscardingOutputFormat()); + } + } + // -------------------------------------------------------------------------------------------- // Pipeline Visitor Methods // -------------------------------------------------------------------------------------------- @@ -63,10 +77,13 @@ public void enterCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == null) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { currentCompositeTransform = transform; - if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { + if (transform instanceof CoGroupByKey && + node.getInput().expand().size() != 2) { // we can only optimize CoGroupByKey for input size 2 currentCompositeTransform = null; } @@ -80,7 +97,9 @@ public void leaveCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == transform) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { LOG.info(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); applyBatchTransform(transform, node, translator); @@ -106,10 +125,13 @@ public void visitTransform(TransformTreeNode node) { // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator == null) { LOG.info(node.getTransform().getClass().toString()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + throw new UnsupportedOperationException( + "The transform " + transform + " is currently not supported."); } applyBatchTransform(transform, node, translator); } @@ -119,7 +141,10 @@ public void visitValue(PValue value, TransformTreeNode producer) { // do nothing here } - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + private > void applyBatchTransform( + PTransform transform, + TransformTreeNode node, + BatchTransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @@ -136,8 +161,8 @@ public void visitValue(PValue value, TransformTreeNode producer) { /** * A translator of a {@link PTransform}. */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); + public interface BatchTransformTranslator { + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } private static String genSpaces(int n) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index a03352efae15..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,81 +17,87 @@ */ 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; import org.apache.beam.sdk.io.Read; 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.GroupReduceFunction; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +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 { @@ -100,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; @@ -221,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); @@ -230,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(); @@ -269,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); + + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(transform.getCoder())); - //inputDataSet.print(); - DataSink dataSink = inputDataSet.writeAsText(filenamePrefix); + DataSink dataSink = valueStream.writeAsText(filenamePrefix); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -282,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; + + if (windowingStrategy.getWindowFn().isNonMerging()) { + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; + + 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()); + } - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); - // Partially GroupReduce the values into the intermediate format VA (combine) - GroupCombineOperator, KV> groupCombine = - new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, - "GroupCombine: " + transform.getName()); + transformSideInputs(transform.getSideInputs(), groupCombine, context); - // Reduce fully to VO - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + TypeInformation>> reduceTypeInfo = + context.getTypeInfo(context.getOutput(transform)); - TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); + Grouping>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); - 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()); - // Fully reduce the values and create output format VO - GroupReduceOperator, KV> 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()) { @@ -432,147 +660,166 @@ 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 - public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { + @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); + 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>() { + @Override + public void flatMap(String s, Collector> collector) throws Exception { + // never return anything + } + }).returns( + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + (Coder) VoidCoder.of(), + GlobalWindow.Coder.INSTANCE))); + } 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); - } - } - 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 71950cf216cb..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,31 +18,40 @@ 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; 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 +64,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; } @@ -68,13 +83,16 @@ public PipelineOptions getPipelineOptions() { } @SuppressWarnings("unchecked") - public DataSet getInputDataSet(PValue value) { - return (DataSet) dataSets.get(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); } - 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); } } @@ -91,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()); - } + @SuppressWarnings("unchecked") + public TypeInformation> getTypeInfo(PCollection collection) { + Coder valueCoder = collection.getCoder(); + WindowedValue.FullWindowedValueCoder windowedValueCoder = + WindowedValue.getFullCoder( + valueCoder, + collection.getWindowingStrategy().getWindowFn().windowCoder()); - public TypeInformation getOutputTypeInfo() { - return getTypeInfo((PValue) currentTransform.getOutput()); + return new CoderTypeInformation<>(windowedValueCoder); } @SuppressWarnings("unchecked") - I getInput(PTransform transform) { - return (I) currentTransform.getInput(); + T getInput(PTransform transform) { + return (T) currentTransform.getInput(); } @SuppressWarnings("unchecked") - O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); + T getOutput(PTransform transform) { + return (T) currentTransform.getOutput(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 618727dae4d9..60a9607a9bb4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,15 +18,13 @@ 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; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper; -import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupByKeyWrapper; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundMultiWrapper; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkParDoBoundWrapper; -import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction; +import org.apache.beam.runners.flink.translation.wrappers.streaming.KVKeySelector; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSink; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; @@ -38,13 +36,13 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; 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; @@ -74,8 +72,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -99,7 +95,6 @@ public class FlinkStreamingTransformTranslators { // here you can find all the available translators. static { - TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); TRANSLATORS.put(Read.Bounded.class, new BoundedReadSourceTranslator()); TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); @@ -122,47 +117,6 @@ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getT // Transformation Implementations // -------------------------------------------------------------------------------------------- - private static class CreateStreamingTranslator implements - FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Create.Values transform, FlinkStreamingTranslationContext context) { - PCollection output = context.getOutput(transform); - Iterable elements = transform.getElements(); - - // we need to serialize the elements to byte arrays, since they might contain - // elements that are not serializable by Java serialization. We deserialize them - // in the FlatMap function using the Coder. - - List serializedElements = Lists.newArrayList(); - Coder elementCoder = output.getCoder(); - for (OUT element: elements) { - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - try { - elementCoder.encode(element, bao, Coder.Context.OUTER); - serializedElements.add(bao.toByteArray()); - } catch (IOException e) { - throw new RuntimeException("Could not serialize Create elements using Coder: " + e); - } - } - - - DataStream initDataSet = context.getExecutionEnvironment().fromElements(1); - - FlinkStreamingCreateFunction createFunction = - new FlinkStreamingCreateFunction<>(serializedElements, elementCoder); - - WindowedValue.ValueOnlyWindowedValueCoder windowCoder = WindowedValue.getValueOnlyCoder(elementCoder); - TypeInformation> outputType = new CoderTypeInformation<>(windowCoder); - - DataStream> outputDataStream = initDataSet.flatMap(createFunction) - .returns(outputType); - - context.setOutputDataStream(output, outputDataStream); - } - } - - private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); @@ -228,29 +182,15 @@ public void translateNode(Read.Bounded transform, FlinkStreamingTranslationCo BoundedSource boundedSource = transform.getSource(); PCollection output = context.getOutput(transform); - Coder defaultOutputCoder = boundedSource.getDefaultOutputCoder(); - CoderTypeInformation typeInfo = new CoderTypeInformation<>(defaultOutputCoder); + TypeInformation> typeInfo = context.getTypeInfo(output); - DataStream source = context.getExecutionEnvironment().createInput( + DataStream> source = context.getExecutionEnvironment().createInput( new SourceInputFormat<>( boundedSource, context.getPipelineOptions()), typeInfo); - DataStream> windowedStream = source.flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(T value, Collector> out) throws Exception { - out.collect( - WindowedValue.of(value, - Instant.now(), - GlobalWindow.INSTANCE, - PaneInfo.NO_FIRING)); - } - }) - .assignTimestampsAndWatermarks(new IngestionTimeExtractor>()); - - context.setOutputDataStream(output, windowedStream); + context.setOutputDataStream(output, source); } } @@ -382,8 +322,8 @@ public void translateNode(GroupByKey transform, FlinkStreamingTranslationC DataStream>> inputDataStream = context.getInputDataStream(input); KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); + KeyedStream>, K> groupByKStream = + KVKeySelector.keyBy(inputDataStream, inputKvCoder); DataStream>>> groupedByKNWstream = FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(), @@ -403,8 +343,8 @@ public void translateNode(Combine.PerKey transform, FlinkStreaming KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); KvCoder outputKvCoder = (KvCoder) context.getOutput(transform).getCoder(); - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); + KeyedStream>, K> groupByKStream = + KVKeySelector.keyBy(inputDataStream, inputKvCoder); Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) transform.getFn(); DataStream>> groupedByKNWstream = diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/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/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/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/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/CombineFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java deleted file mode 100644 index e5567d3ea3b2..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.wrappers; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; - -import com.google.common.collect.Lists; - -import org.apache.flink.api.common.accumulators.Accumulator; - -import java.io.Serializable; - -/** - * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn} - * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using - * the combine function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} - * operation. - */ -public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { - - private AA aa; - private Combine.CombineFn combiner; - - public CombineFnAggregatorWrapper() { - } - - public CombineFnAggregatorWrapper(Combine.CombineFn combiner) { - this.combiner = combiner; - this.aa = combiner.createAccumulator(); - } - - @Override - public void add(AI value) { - combiner.addInput(aa, value); - } - - @Override - public Serializable getLocalValue() { - return (Serializable) combiner.extractOutput(aa); - } - - @Override - public void resetLocal() { - aa = combiner.createAccumulator(); - } - - @Override - @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); - } - - @Override - public Accumulator clone() { - // copy it by merging - AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa)); - CombineFnAggregatorWrapper result = new - CombineFnAggregatorWrapper<>(combiner); - result.aa = aaCopy; - return result; - } - - @Override - public void addValue(AI value) { - add(value); - } - - @Override - public String getName() { - return "CombineFn: " + combiner.toString(); - } - - @Override - public Combine.CombineFn getCombineFn() { - return combiner; - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index eb32fa2fd74a..82d3fb8ffae3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -33,20 +33,21 @@ * the function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} * operation. */ -public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { +public class SerializableFnAggregatorWrapper + implements Aggregator, Accumulator { - private AO aa; - private Combine.CombineFn combiner; + private OutputT aa; + private Combine.CombineFn combiner; - public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { + public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { this.combiner = combiner; resetLocal(); } - + @Override @SuppressWarnings("unchecked") - public void add(AI value) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, value)); + public void add(InputT value) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, value)); } @Override @@ -56,17 +57,17 @@ public Serializable getLocalValue() { @Override public void resetLocal() { - this.aa = combiner.apply(ImmutableList.of()); + this.aa = combiner.apply(ImmutableList.of()); } @Override @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); + public void merge(Accumulator other) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, (InputT) other.getLocalValue())); } @Override - public void addValue(AI value) { + public void addValue(InputT value) { add(value); } @@ -76,15 +77,15 @@ public String getName() { } @Override - public Combine.CombineFn getCombineFn() { + public Combine.CombineFn getCombineFn() { return combiner; } @Override - public Accumulator clone() { + public Accumulator clone() { // copy it by merging - AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); - SerializableFnAggregatorWrapper result = new + OutputT resultCopy = combiner.apply(Lists.newArrayList((InputT) aa)); + SerializableFnAggregatorWrapper result = new SerializableFnAggregatorWrapper<>(combiner); result.aa = resultCopy; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java index 2766a873e35f..28a4b9ee59df 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java @@ -22,6 +22,7 @@ import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; @@ -31,10 +32,11 @@ import java.lang.reflect.Field; /** - * Wrapper class to use generic Write.Bound transforms as sinks. + * Wrapper for executing a {@link Sink} on Flink as an {@link OutputFormat}. + * * @param The type of the incoming records. */ -public class SinkOutputFormat implements OutputFormat { +public class SinkOutputFormat implements OutputFormat> { private final Sink sink; @@ -88,9 +90,9 @@ public void open(int taskNumber, int numTasks) throws IOException { } @Override - public void writeRecord(T record) throws IOException { + public void writeRecord(WindowedValue record) throws IOException { try { - writer.write(record); + writer.write(record.getValue()); } catch (Exception e) { throw new IOException("Couldn't write record.", e); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index debd1a14d525..1d06b1ac2fc9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -21,12 +21,16 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplitAssigner; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,10 +39,10 @@ /** - * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a - * Dataflow {@link org.apache.beam.sdk.io.Source}. + * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. */ -public class SourceInputFormat implements InputFormat> { +public class SourceInputFormat + implements InputFormat, SourceInputSplit> { private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); private final BoundedSource initialSource; @@ -122,12 +126,16 @@ public boolean reachedEnd() throws IOException { } @Override - public T nextRecord(T t) throws IOException { + public WindowedValue nextRecord(WindowedValue t) throws IOException { if (inputAvailable) { final T current = reader.getCurrent(); + final Instant timestamp = reader.getCurrentTimestamp(); // advance reader to have a record ready next time inputAvailable = reader.advance(); - return current; + return WindowedValue.of( + current, + timestamp, + GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } return null; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index bb6ed67cb29f..26532a47f47b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -197,14 +197,14 @@ protected Aggregator createAggreg } protected void checkTimestamp(WindowedValue ref, Instant timestamp) { - if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { - throw new IllegalArgumentException(String.format( - "Cannot output with timestamp %s. Output timestamps must be no earlier than the " - + "timestamp of the current input (%s) minus the allowed skew (%s). See the " - + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", - timestamp, ref.getTimestamp(), - PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); - } +// if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { +// throw new IllegalArgumentException(String.format( +// "Cannot output with timestamp %s. Output timestamps must be no earlier than the " +// + "timestamp of the current input (%s) minus the allowed skew (%s). See the " +// + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", +// timestamp, ref.getTimestamp(), +// PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); +// } } protected WindowedValue makeWindowedValue( @@ -267,4 +267,4 @@ public abstract WindowingInternals windowingInternalsHelper( WindowedValue inElement, Collector> outCollector); -} \ No newline at end of file +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KVKeySelector.java similarity index 55% rename from runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java rename to runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KVKeySelector.java index 3bf566bce762..2fa6f10758fd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KVKeySelector.java @@ -18,10 +18,8 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -32,36 +30,39 @@ import org.apache.flink.streaming.api.datastream.KeyedStream; /** - * This class groups the elements by key. It assumes that already the incoming stream - * is composed of [Key,Value] pairs. - * */ -public class FlinkGroupByKeyWrapper { + * {@link KeySelector} for keying a {@link org.apache.beam.sdk.values.PCollection} + * of {@code KV}. + */ +public class KVKeySelector + implements KeySelector>, K>, ResultTypeQueryable { + + private final TypeInformation keyTypeInfo; - /** - * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement - * multiple interfaces. - */ - private interface KeySelectorWithQueryableResultType extends KeySelector>, K>, ResultTypeQueryable { + private KVKeySelector(TypeInformation keyTypeInfo) { + this.keyTypeInfo = keyTypeInfo; } - public static KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { - final Coder keyCoder = inputKvCoder.getKeyCoder(); - final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); - final boolean isKeyVoid = keyCoder instanceof VoidCoder; + @Override + @SuppressWarnings("unchecked") + public K getKey(WindowedValue> value) throws Exception { + K key = value.getValue().getKey(); + // hack, because Flink does not allow null keys + return key != null ? key : (K) new Integer(0); + } - return inputDataStream.keyBy( - new KeySelectorWithQueryableResultType() { + @Override + public TypeInformation getProducedType() { + return keyTypeInfo; + } - @Override - public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : - value.getValue().getKey(); - } + public static KeyedStream>, K> keyBy( + DataStream>> inputDataStream, + KvCoder inputKvCoder) { - @Override - public TypeInformation getProducedType() { - return keyTypeInfo; - } - }); + final Coder keyCoder = inputKvCoder.getKeyCoder(); + final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); + + return inputDataStream.keyBy(new KVKeySelector(keyTypeInfo)); } + } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java deleted file mode 100644 index d6aff7d7a4ee..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.wrappers.streaming.io; - -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.util.Collector; -import org.joda.time.Instant; - -import java.io.ByteArrayInputStream; -import java.util.List; - -/** - * This flat map function bootstraps from collection elements and turns them into WindowedValues - * (as required by the Flink runner). - */ -public class FlinkStreamingCreateFunction implements FlatMapFunction> { - - private final List elements; - private final Coder coder; - - public FlinkStreamingCreateFunction(List elements, Coder coder) { - this.elements = elements; - this.coder = coder; - } - - @Override - public void flatMap(IN value, Collector> out) throws Exception { - - @SuppressWarnings("unchecked") - OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; - for (byte[] element : elements) { - ByteArrayInputStream bai = new ByteArrayInputStream(element); - OUT outValue = coder.decode(bai, Coder.Context.OUTER); - - if (outValue == null) { - out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } else { - out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } - } - - out.close(); - } -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/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/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()); } 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/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java index f3ceba76077a..a488768467ea 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java @@ -43,6 +43,7 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestHarnessUtil; import org.joda.time.Duration; import org.joda.time.Instant; @@ -52,7 +53,7 @@ import java.util.Comparator; import java.util.concurrent.ConcurrentLinkedQueue; -public class GroupAlsoByWindowTest { +public class GroupAlsoByWindowTest extends StreamingMultipleProgramsTestBase { private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index 1b55c61bacf3..c8f970588315 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -29,15 +29,18 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; import com.google.common.base.Joiner; +import com.google.common.collect.Lists; import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; import java.io.Serializable; -import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collections; public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { @@ -61,63 +64,51 @@ protected void postSubmit() throws Exception { compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); } - public static class ExtractUserAndTimestamp extends DoFn, String> { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - KV record = c.element(); - int timestamp = record.getKey(); - String userName = record.getValue(); - if (userName != null) { - // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp)); - } - } - } - @Override protected void testProgram() throws Exception { Pipeline p = FlinkTestPipeline.createForStreaming(); - PCollection output = - p.apply(Create.of(Arrays.asList( - KV.of(0, "user1"), - KV.of(1, "user1"), - KV.of(2, "user1"), - KV.of(10, "user2"), - KV.of(1, "user2"), - KV.of(15000, "user2"), - KV.of(12000, "user2"), - KV.of(25000, "user3")))) - .apply(ParDo.of(new ExtractUserAndTimestamp())) - .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) - .triggering(AfterWatermark.pastEndOfWindow()) - .withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()) - - .apply(ParDo.of(new DoFn>() { - @Override - public void processElement(ProcessContext c) throws Exception { - String elem = c.element(); - c.output(KV.of((Void) null, elem)); - } - })) - .apply(GroupByKey.create()) - .apply(ParDo.of(new DoFn>, String>() { - @Override - public void processElement(ProcessContext c) throws Exception { - KV> elem = c.element(); - StringBuilder str = new StringBuilder(); - str.append("k: " + elem.getKey() + " v:"); - for (String v : elem.getValue()) { - str.append(" " + v); - } - c.output(str.toString()); + PCollection output = p + .apply(Create.timestamped( + TimestampedValue.of("user1", new Instant(0)), + TimestampedValue.of("user1", new Instant(1)), + TimestampedValue.of("user1", new Instant(2)), + TimestampedValue.of("user2", new Instant(10)), + TimestampedValue.of("user2", new Instant(1)), + TimestampedValue.of("user2", new Instant(15000)), + TimestampedValue.of("user2", new Instant(12000)), + TimestampedValue.of("user3", new Instant(25000)))) + .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) + .triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) + + .apply(ParDo.of(new DoFn>() { + @Override + public void processElement(ProcessContext c) throws Exception { + String elem = c.element(); + c.output(KV.of(null, elem)); + } + })) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV> elem = c.element(); + ArrayList strings = Lists.newArrayList(elem.getValue()); + Collections.sort(strings); + StringBuilder str = new StringBuilder(); + str.append("k: " + elem.getKey() + " v:"); + for (String v : strings) { + str.append(" " + v); } - })); + c.output(str.toString()); + } + })); + output.apply(TextIO.Write.to(resultPath)); + p.run(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 1265acdd9bd4..c2cd598cce0f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Aggregator; @@ -593,7 +593,7 @@ public PDone apply(PBegin input) { final PCollectionView actual = input.apply("CreateActual", createActual); input - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply(Create.of(0).withCoder(VarIntCoder.of())) .apply(ParDo.named("RunChecks").withSideInputs(actual) .of(new CheckerDoFn<>(checkerFn, actual))); @@ -604,8 +604,11 @@ public PDone apply(PBegin input) { /** * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of * a {@link PCollectionView}, and adjusts counters and thrown exceptions for use in testing. + * + *

The input is ignored, but is {@link Integer} to be usable on runners that do not support + * null values. */ - private static class CheckerDoFn extends DoFn { + private static class CheckerDoFn extends DoFn { private final SerializableFunction checkerFn; private final Aggregator success = createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn()); @@ -669,14 +672,17 @@ public PDone apply(PBegin input) { final PCollectionView expected = input.apply("CreateExpected", createExpected); input - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) - .apply(ParDo.named("RunChecks").withSideInputs(actual, expected) + .apply(Create.of(0).withCoder(VarIntCoder.of())) + .apply("RunChecks", ParDo.withSideInputs(actual, expected) .of(new CheckerDoFn<>(relation, actual, expected))); return PDone.in(input.getPipeline()); } - private static class CheckerDoFn extends DoFn { + /** + * Input is ignored, but is {@link Integer} for runners that do not support null values. + */ + private static class CheckerDoFn extends DoFn { private final Aggregator success = createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn()); private final Aggregator failure = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java new file mode 100644 index 000000000000..aa291d485699 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresFixedWindows.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing.capabilities; + +import org.apache.beam.sdk.transforms.windowing.FixedWindows; + +/** + * Category tag indicating the test requires support for {@link FixedWindows}. + */ +public interface RequiresFixedWindows {} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java new file mode 100644 index 000000000000..f19e7c4ba8d3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/capabilities/RequiresTimestampControl.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing.capabilities; + +/** + * Category tag indicating the test requires support for controlling timestamps + * of elements. + */ +public interface RequiresTimestampControl {} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 8a39c98a16e7..683b6a0b7ddc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; @@ -1354,17 +1353,17 @@ public Globally withSideInputs( @Override public PCollection apply(PCollection input) { - PCollection> withKeys = input - .apply(WithKeys.of((Void) null)) - .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); + PCollection> withKeys = input + .apply(WithKeys.of(1)) + .setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder())); - Combine.PerKey combine = + Combine.PerKey combine = Combine.fewKeys(fn.asKeyedFn(), fnDisplayData); if (!sideInputs.isEmpty()) { combine = combine.withSideInputs(sideInputs); } - PCollection> combined; + PCollection> combined; if (fanout >= 2) { combined = withKeys.apply(combine.withHotKeyFanout(fanout)); } else { @@ -1398,11 +1397,11 @@ private PCollection insertDefaultValueIfEmpty(PCollection mayb final OutputT defaultValue = fn.defaultValue(); PCollection defaultIfEmpty = maybeEmpty.getPipeline() - .apply("CreateVoid", Create.of((Void) null).withCoder(VoidCoder.of())) + .apply("CreateVoid", Create.of(1).withCoder(VarIntCoder.of())) .apply(ParDo.named("ProduceDefault").withSideInputs(maybeEmptyView).of( - new DoFn() { + new DoFn() { @Override - public void processElement(DoFn.ProcessContext c) { + public void processElement(DoFn.ProcessContext c) { Iterator combined = c.sideInput(maybeEmptyView).iterator(); if (!combined.hasNext()) { c.output(defaultValue); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java index 58188def4226..9c221dcfdea1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; @@ -148,7 +148,7 @@ public PCollection apply(PCollection in) { PCollectionView> iterableView = in.apply(View.asIterable()); return in.getPipeline() - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply(Create.of(1).withCoder(VarIntCoder.of())) .apply(ParDo .withSideInputs(iterableView) .of(new SampleAnyDoFn<>(limit, iterableView))) @@ -165,7 +165,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * A {@link DoFn} that returns up to limit elements from the side input PCollection. */ - private static class SampleAnyDoFn extends DoFn { + private static class SampleAnyDoFn extends DoFn { long limit; final PCollectionView> iterableView; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 43f1adf0c0c8..587c041d0654 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -35,7 +35,7 @@ /** * A UnionCoder encodes RawUnionValues. */ -class UnionCoder extends StandardCoder { +public class UnionCoder extends StandardCoder { // TODO: Think about how to integrate this with a schema object (i.e. // a tuple of tuple tags). /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java index 8c87c2632655..1e6cb07d2540 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -26,6 +27,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; @@ -125,7 +127,7 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testUnboundedInputTimestamps() { Pipeline p = TestPipeline.create(); long numElements = 1000; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java index a261fb274459..8840df5860a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; @@ -136,7 +137,7 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testUnboundedSourceTimestamps() { Pipeline p = TestPipeline.create(); long numElements = 1000; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index e491feadd2d4..69879c6d8624 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl; import org.apache.beam.sdk.transforms.Create.Values.CreateSource; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; @@ -233,7 +234,7 @@ public void processElement(ProcessContext c) { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testCreateTimestamped() { Pipeline p = TestPipeline.create(); @@ -252,7 +253,7 @@ public void testCreateTimestamped() { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresTimestampControl.class}) public void testCreateTimestampedEmpty() { Pipeline p = TestPipeline.create(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 042b864b2678..fb5fab5b44e5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; @@ -128,8 +128,8 @@ public void testEmptyFlattenAsSideInput() { .apply(View.asIterable()); PCollection output = p - .apply(Create.of((Void) null).withCoder(VoidCoder.of())) - .apply(ParDo.withSideInputs(view).of(new DoFn() { + .apply(Create.of(1).withCoder(VarIntCoder.of())) + .apply(ParDo.withSideInputs(view).of(new DoFn() { @Override public void processElement(ProcessContext c) { for (String side : c.sideInput(view)) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 9193267c2800..6c1cd9a0f1ab 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows; import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess; import org.apache.beam.sdk.transforms.ParDo.Bound; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -1347,7 +1348,7 @@ public Void apply(Iterable input) { } @Test - @Category(RunnableOnService.class) + @Category({RunnableOnService.class, RequiresFixedWindows.class}) public void testWindowingInStartAndFinishBundle() { Pipeline pipeline = TestPipeline.create();